commit 4d998c12c95a6e5ce3d66c0d861e75e33b5cf013 Author: joewitt Date: Mon Dec 8 15:22:14 2014 -0500 Initial code contribution diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000000..21c67f215d --- /dev/null +++ b/.gitignore @@ -0,0 +1,9 @@ +target +.project +.settings +nbactions.xml + +# Intellij +.idea/ +*.iml +*.iws diff --git a/DISCLAIMER b/DISCLAIMER new file mode 100644 index 0000000000..0f8e7a1ecc --- /dev/null +++ b/DISCLAIMER @@ -0,0 +1,15 @@ +Apache NiFi is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator PMC. + +Incubation is required of all newly accepted projects until a further review +indicates that the infrastructure, communications, and decision making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness +or stability of the code, it does indicate that the project has yet to be +fully endorsed by the ASF. + +For more information about the incubation status of the Apache NiFi project +you can go to the following page: + +http://nifi.incubator.apache.org/ diff --git a/LICENSE b/LICENSE new file mode 100644 index 0000000000..d645695673 --- /dev/null +++ b/LICENSE @@ -0,0 +1,202 @@ + + 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. diff --git a/NOTICE b/NOTICE new file mode 100644 index 0000000000..e02ae44d1d --- /dev/null +++ b/NOTICE @@ -0,0 +1,7 @@ +Apache NiFi +Copyright 2014 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +Please see LICENSE for additional copyright and licensing information. diff --git a/README.md b/README.md new file mode 100644 index 0000000000..3f051341b5 --- /dev/null +++ b/README.md @@ -0,0 +1,65 @@ +# Apache NiFi + +Apache NiFi is a dataflow system based on the concepts of flow-based programming. It is currently apart of the Apache Incubator. + +## Table of Contents + +- [Features](#features) +- [Getting Started](#getting-started) +- [Getting Help](#getting-help) +- [Requirements](#requirements) +- [License](#license) +- [Disclaimer](#disclaimer) + +## Features + +Apache NiFi supports powerful and scalable directed graphs of data routing, transformation, and system mediation logic. Some of the high-level capabilities and objectives of Apache NiFi include: + +- Web-based user interface for seamless experience between design, control, feedback, and monitoring of data flows +- Highly configurable along several dimensions of quality of service such as loss tolerant versus guaranteed delivery, low latency versus high throughput, and priority based queuing +- Fine-grained data provenance for all data received, forked, joined, cloned, modified, sent, and ultimately dropped as data reaches its configured end-state +- Component-based extension model along well defined interfaces enabling rapid development and effective testing + +## Getting Started + +Execute /bin/nifi.sh + +## Getting Help +If you have questions, you can reach out to our mailing list: dev@nifi.incubator.apache.org +([archive](http://mail-archives.apache.org/mod_mbox/incubator-nifi-dev)). +We're also often available in IRC: #nifi on +[irc.freenode.net](http://webchat.freenode.net/?channels=#nifi). + + +## Requirements +* JDK 1.7 or higher + +## License + +Except as otherwise noted this software is licensed under the +[Apache License, Version 2.0](http://www.apache.org/licenses/LICENSE-2.0.html) + +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. + +## Disclaimer + +Apache NiFi is an effort undergoing incubation at the Apache Software +Foundation (ASF), sponsored by the Apache Incubator PMC. + +Incubation is required of all newly accepted projects until a further review +indicates that the infrastructure, communications, and decision making process +have stabilized in a manner consistent with other successful ASF projects. + +While incubation status is not necessarily a reflection of the completeness +or stability of the code, it does indicate that the project has yet to be +fully endorsed by the ASF. diff --git a/assemblies/nifi/pom.xml b/assemblies/nifi/pom.xml new file mode 100644 index 0000000000..ec06a2327d --- /dev/null +++ b/assemblies/nifi/pom.xml @@ -0,0 +1,485 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi + 0.0.1-SNAPSHOT + pom + + NiFi Release + This is the assembly for the Main Standard Release of NiFi + + + + + maven-dependency-plugin + + + unpack-shared-resources + + unpack-dependencies + + generate-resources + + ${project.build.directory}/generated-resources + nifi-resources + org.apache.nifi + false + + + + + + maven-assembly-plugin + + + src/main/assembly/dependencies.xml + + + + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + + exec + + + + + ${project.build.directory}/${project.artifactId}-${project.version}-${nifi.assembly.id}.dir/${nifi.executable} + + + + + + ${basedir}/src/main/resources + + + ${project.build.directory}/generated-resources + true + + + + + + ch.qos.logback + logback-classic + compile + + + org.slf4j + jcl-over-slf4j + compile + + + org.slf4j + jul-to-slf4j + compile + + + org.slf4j + log4j-over-slf4j + ${org.slf4j.version} + compile + + + org.slf4j + slf4j-api + compile + + + org.apache.nifi + nifi-api + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-runtime + ${framework.version} + + + org.apache.nifi + nifi-resources + ${framework.version} + resources + provided + zip + + + org.apache.nifi + nifi-framework-nar + ${framework.version} + nar + + + org.apache.nifi + volatile-provenance-repository-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + persistent-provenance-repository-nar + 0.0.1-SNAPSHOT + nar + + + + org.apache.nifi + standard-services-api-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + ssl-context-service-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + distributed-cache-services-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + nifi-standard-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + nifi-jetty-bundle + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + update-attribute-nar + 0.0.1-SNAPSHOT + nar + + + + org.apache.nifi + monitor-threshold-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + hadoop-libraries-nar + 0.0.1-SNAPSHOT + nar + + + org.apache.nifi + hadoop-nar + 0.0.1-SNAPSHOT + nar + + + + + + 0.0.1-SNAPSHOT + + + 256 + 512 + 128 + 128 + 10m + 10 + + + true + 10 sec + 500 ms + 30 sec + + ./conf/flow.xml.gz + ./conf/archive/ + ./conf/reporting-tasks.xml + ./conf/controller-services.xml + ./conf/authority-providers.xml + ./conf/templates + ./database_repository + + org.apache.nifi.controller.repository.WriteAheadFlowFileRepository + ./flowfile_repository + 256 + 2 mins + false + org.apache.nifi.controller.FileSystemSwapManager + 20000 + ./flowfile_repository/swap + 5 sec + 1 + 5 sec + 4 + + org.apache.nifi.controller.repository.FileSystemRepository + 10 MB + 100 + ./content_repository + + + false + false + + + + + + 30 sec + ./lib + ./work/nar/ + ./work/docs/components + + PBEWITHMD5AND256BITAES-CBC-OPENSSL + BC + ;LOCK_TIMEOUT=25000;WRITE_DELAY=0;AUTO_SERVER=FALSE + + 9990 + + + org.apache.nifi.provenance.PersistentProvenanceRepository + ./provenance_repository + 24 hours + 1 GB + 5 mins + 100 MB + 2 + true + EventType, FlowFileUUID, Filename, ProcessorID + + 500 MB + false + 16 + + + 100000 + + + org.apache.nifi.controller.status.history.VolatileComponentStatusRepository + 288 + 5 mins + + + ./lib + + 8080 + + + ./work/jetty + + + + + + + + + + + ./conf/authorized-users.xml + 24 hours + file-provider + + + + + + + 5 sec + false + 30 sec + 45 sec + false + + + 500 ms + 3 + 1 sec + + + false + + + 2 + + + + + false + + + + 10 + 30 sec + 30 sec + 10 + 5 sec + 10 + 0 sec + + + bin + ${project.artifactId}-${project.version}/bin/${project.artifactId} + + + + + standard-rpm + + false + + + + + org.codehaus.mojo + rpm-maven-plugin + 2.0.1 + + + + rpm + + + + + ${project.artifactId} + ${project.version} + NiFi data flow framework + + Nifi is a simple event processing framework largely + based on the Flow-Based Programming concepts. + + NONE + http://nifi.apache.org + Utilities + + jdk + + /opt/${project.artifactId} + + _use_internal_dependency_generator 0 + + 750 + 640 + dataflow + dataflow + + + /opt/${project.artifactId}/${project.artifactId}-${project.version} + + + /opt/${project.artifactId}/${project.artifactId}-${project.version}/bin + 750 + + + ${project.build.directory}/generated-resources/bin/nifi + ${project.artifactId} + true + + + ${project.build.directory}/generated-resources/bin/wrapper-linux-x86-32 + + + ${project.build.directory}/generated-resources/bin/wrapper-linux-x86-64 + + + + + /opt/${project.artifactId}/${project.artifactId}-${project.version}/conf + true + + + ${project.build.directory}/generated-resources/conf + + nifi.properties + + true + + + ${project.build.directory}/generated-resources/conf/nifi.properties + ${project.artifactId}.properties + true + + + + + /opt/${project.artifactId}/${project.artifactId}-${project.version}/lib + + + + ${project.build.directory}/generated-resources/lib + + **/*-aix* + **/*-freebsd* + **/*-hpux* + **/*-linux-ia* + **/*-linux-ppc* + **/*-macosx* + **/*-solaris* + **/*-windows* + + + + + + /opt/${project.artifactId}/${project.artifactId}-${project.version}/docs + + + ${project.build.directory}/generated-resources/docs + + + + + /opt/${project.artifactId}/${project.artifactId}-${project.version}/logs + + + ${project.build.directory}/generated-resources/logs + + + + + + + + + + + diff --git a/assemblies/nifi/src/main/assembly/dependencies.xml b/assemblies/nifi/src/main/assembly/dependencies.xml new file mode 100644 index 0000000000..339275bd11 --- /dev/null +++ b/assemblies/nifi/src/main/assembly/dependencies.xml @@ -0,0 +1,123 @@ + + + + bin + + dir + tar.gz + + true + ${project.artifactId}-${project.version} + + + + runtime + lib + true + + + + + + ${project.build.directory}/generated-resources/bin + bin + 0750 + 0750 + false + + + ${project.build.directory}/generated-resources/conf + conf + + nifi.properties + + 0750 + 0640 + true + + + ${project.build.directory}/generated-resources/lib + lib + 0750 + 0640 + + + ${project.build.directory}/generated-resources/logs + logs + 0750 + 0640 + + + ${project.build.directory}/generated-resources/docs + docs + 0750 + 0640 + false + + + + + + ${project.build.directory}/generated-resources/conf/nifi.properties + conf + ${project.artifactId}.properties + 0640 + true + + + ${project.build.directory}/generated-resources/bin/nifi.sh + bin + nifi.sh + 0750 + true + + + ${project.build.directory}/generated-resources/docs/README.md + docs + README.md + 0640 + true + + + ${project.build.directory}/generated-resources/DISCLAIMER + ./ + DISCLAIMER + 0640 + true + + + ${project.build.directory}/generated-resources/DISCLAIMER + ./ + DISCLAIMER + 0640 + true + + + ${project.build.directory}/generated-resources/LICENSE + ./ + LICENSE + 0640 + true + + + ${project.build.directory}/generated-resources/NOTICE + ./ + NOTICE + 0640 + true + + + diff --git a/commons/core-flowfile-attributes/pom.xml b/commons/core-flowfile-attributes/pom.xml new file mode 100644 index 0000000000..d611134c69 --- /dev/null +++ b/commons/core-flowfile-attributes/pom.xml @@ -0,0 +1,32 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-core-flowfile-attributes + 0.0.1-SNAPSHOT + jar + + NiFi: Core FlowFile Attributes + + + + diff --git a/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java new file mode 100644 index 0000000000..24f43ca587 --- /dev/null +++ b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/CoreAttributes.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.flowfile.attributes; + +public enum CoreAttributes implements FlowFileAttributeKey { + /** + * The flowfile's path indicates the relative directory to which a FlowFile belongs and does not + * contain the filename + */ + PATH("path"), + + /** + * The flowfile's absolute path indicates the absolute directory to which a FlowFile belongs and does not + * contain the filename + */ + ABSOLUTE_PATH("absolute.path"), + + /** + * The filename of the FlowFile. The filename should not contain any directory structure. + */ + FILENAME("filename"), + + /** + * A unique UUID assigned to this FlowFile + */ + UUID("uuid"), + + /** + * A numeric value indicating the FlowFile priority + */ + PRIORITY("priority"), + + /** + * The MIME Type of this FlowFile + */ + MIME_TYPE("mime.type"), + + /** + * Specifies the reason that a FlowFile is being discarded + */ + DISCARD_REASON("discard.reason"), + + /** + * Indicates an identifier other than the FlowFile's UUID that is known to refer to this FlowFile. + */ + ALTERNATE_IDENTIFIER("alternate.identifier"); + + private final String key; + private CoreAttributes(final String key) { + this.key = key; + } + + @Override + public String key() { + return key; + } + +} diff --git a/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java new file mode 100644 index 0000000000..cc6c28ebaa --- /dev/null +++ b/commons/core-flowfile-attributes/src/main/java/org/apache/nifi/flowfile/attributes/FlowFileAttributeKey.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.flowfile.attributes; + +public interface FlowFileAttributeKey { + String key(); +} diff --git a/commons/data-provenance-utils/.gitignore b/commons/data-provenance-utils/.gitignore new file mode 100755 index 0000000000..19f2e002ce --- /dev/null +++ b/commons/data-provenance-utils/.gitignore @@ -0,0 +1,2 @@ +/target +/target diff --git a/commons/data-provenance-utils/pom.xml b/commons/data-provenance-utils/pom.xml new file mode 100644 index 0000000000..0f33c06b28 --- /dev/null +++ b/commons/data-provenance-utils/pom.xml @@ -0,0 +1,42 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + data-provenance-utils + 0.0.1-SNAPSHOT + jar + + data-provenance-utils + + + + org.apache.nifi + nifi-api + [0.0.1-SNAPSHOT,1.0.0-SNAPSHOT) + + + org.apache.nifi + nifi-core-flowfile-attributes + 0.0.1-SNAPSHOT + + + diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java new file mode 100644 index 0000000000..dc24a93fe4 --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncLineageSubmission.java @@ -0,0 +1,87 @@ +/* + * 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.provenance; + +import java.util.Collection; +import java.util.Date; +import java.util.UUID; + +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; +import org.apache.nifi.provenance.lineage.LineageComputationType; + +/** + * + */ +public class AsyncLineageSubmission implements ComputeLineageSubmission { + private final String lineageIdentifier = UUID.randomUUID().toString(); + private final Date submissionTime = new Date(); + + private final LineageComputationType computationType; + private final Long eventId; + private final Collection lineageFlowFileUuids; + + private volatile boolean canceled = false; + + private final StandardLineageResult result; + + public AsyncLineageSubmission(final LineageComputationType computationType, final Long eventId, final Collection lineageFlowFileUuids, final int numSteps) { + this.computationType = computationType; + this.eventId = eventId; + this.lineageFlowFileUuids = lineageFlowFileUuids; + this.result = new StandardLineageResult(numSteps, lineageFlowFileUuids); + } + + @Override + public StandardLineageResult getResult() { + return result; + } + + @Override + public Date getSubmissionTime() { + return submissionTime; + } + + @Override + public String getLineageIdentifier() { + return lineageIdentifier; + } + + @Override + public void cancel() { + this.canceled = true; + } + + @Override + public boolean isCanceled() { + return canceled; + } + + @Override + public LineageComputationType getLineageComputationType() { + return computationType; + } + + @Override + public Long getExpandedEventId() { + return eventId; + } + + @Override + public Collection getLineageFlowFileUuids() { + return lineageFlowFileUuids; + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java new file mode 100644 index 0000000000..42444762be --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/AsyncQuerySubmission.java @@ -0,0 +1,81 @@ +/* + * 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.provenance; + +import java.util.Date; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.search.QuerySubmission; + +/** + * + */ +public class AsyncQuerySubmission implements QuerySubmission { + + public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(60, TimeUnit.SECONDS); + + private final Date submissionTime = new Date(); + private final Query query; + + private volatile boolean canceled = false; + private final StandardQueryResult queryResult; + + /** + * Constructs an AsyncQuerySubmission with the given query and the given + * number of steps, indicating how many results must be added to this + * AsyncQuerySubmission before it is considered finished + * + * @param query + * @param numSteps + */ + public AsyncQuerySubmission(final Query query, final int numSteps) { + this.query = query; + queryResult = new StandardQueryResult(query, numSteps); + } + + @Override + public Date getSubmissionTime() { + return submissionTime; + } + + @Override + public String getQueryIdentifier() { + return query.getIdentifier(); + } + + @Override + public void cancel() { + this.canceled = true; + queryResult.cancel(); + } + + @Override + public boolean isCanceled() { + return canceled; + } + + @Override + public Query getQuery() { + return query; + } + + @Override + public StandardQueryResult getResult() { + return queryResult; + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java new file mode 100644 index 0000000000..dc2903f377 --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/NamedSearchableField.java @@ -0,0 +1,95 @@ +/* + * 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.provenance; + +import org.apache.nifi.provenance.search.SearchableField; +import org.apache.nifi.provenance.search.SearchableFieldType; + +import static java.util.Objects.requireNonNull; + +/** + * + */ +public class NamedSearchableField implements SearchableField { + + private final String identifier; + private final String searchableName; + private final SearchableFieldType fieldType; + private final String friendlyName; + private final boolean attribute; + + NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute) { + this(identifier, searchableName, friendlyName, attribute, SearchableFieldType.STRING); + } + + NamedSearchableField(final String identifier, final String searchableName, final String friendlyName, final boolean attribute, final SearchableFieldType fieldType) { + this.identifier = requireNonNull(identifier); + this.searchableName = requireNonNull(searchableName); + this.friendlyName = requireNonNull(friendlyName); + this.attribute = requireNonNull(attribute); + this.fieldType = requireNonNull(fieldType); + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public String getSearchableFieldName() { + return searchableName; + } + + @Override + public String getFriendlyName() { + return friendlyName; + } + + @Override + public boolean isAttribute() { + return attribute; + } + + @Override + public SearchableFieldType getFieldType() { + return fieldType; + } + + @Override + public String toString() { + return friendlyName; + } + + @Override + public int hashCode() { + return 298347 + searchableName.hashCode() + (attribute ? 1 : 0); + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + + if (!(obj instanceof SearchableField)) { + return false; + } + + final SearchableField other = (SearchableField) obj; + return (this.searchableName.equals(other.getSearchableFieldName()) && attribute == other.isAttribute()); + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java new file mode 100644 index 0000000000..6a934b1ab5 --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFieldParser.java @@ -0,0 +1,53 @@ +/* + * 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.provenance; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.provenance.search.SearchableField; + +public class SearchableFieldParser { + + public static List extractSearchableFields(final String indexedFieldString, final boolean predefinedField) { + final List searchableFields = new ArrayList<>(); + if (indexedFieldString != null) { + final String[] split = indexedFieldString.split(","); + for (String fieldName : split) { + fieldName = fieldName.trim(); + if (fieldName.isEmpty()) { + continue; + } + + final SearchableField searchableField; + if (predefinedField) { + searchableField = SearchableFields.getSearchableField(fieldName); + } else { + searchableField = SearchableFields.newSearchableAttribute(fieldName); + } + + if (searchableField == null) { + throw new RuntimeException("Invalid Configuration: Provenance Repository configured to Index field '" + fieldName + "', but this is not a valid field"); + } + searchableFields.add(searchableField); + } + } + + return searchableFields; + } + +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java new file mode 100644 index 0000000000..97c988029a --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/SearchableFields.java @@ -0,0 +1,84 @@ +/* + * 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.provenance; + +import org.apache.nifi.provenance.search.SearchableField; +import org.apache.nifi.provenance.search.SearchableFieldType; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * + */ +public class SearchableFields { + + public static final SearchableField Identifier = new NamedSearchableField("Identifier", "identifier", "Identifier", false); + public static final SearchableField EventTime = new NamedSearchableField("EventTime", "time", "Event Time", false, SearchableFieldType.DATE); + public static final SearchableField FlowFileUUID = new NamedSearchableField("FlowFileUUID", "uuid", "FlowFile UUID", false); + public static final SearchableField Filename = new NamedSearchableField("Filename", "filename", "Filename", false); + public static final SearchableField EventType = new NamedSearchableField("EventType", "eventType", "Event Type", false); + public static final SearchableField TransitURI = new NamedSearchableField("TransitURI", "transitUri", "Transit URI", false); + public static final SearchableField ComponentID = new NamedSearchableField("ProcessorID", "processorId", "Component ID", false); + public static final SearchableField AlternateIdentifierURI = new NamedSearchableField("AlternateIdentifierURI", "alternateIdentifierUri", "Alternate Identifier URI", false); + public static final SearchableField FileSize = new NamedSearchableField("FileSize", "fileSize", "File Size", false, SearchableFieldType.DATA_SIZE); + public static final SearchableField Details = new NamedSearchableField("Details", "details", "Details", false, SearchableFieldType.STRING); + public static final SearchableField Relationship = new NamedSearchableField("Relationship", "relationship", "Relationship", false, SearchableFieldType.STRING); + + public static final SearchableField LineageStartDate = new NamedSearchableField("LineageStartDate", "lineageStartDate", "Lineage Start Date", false, SearchableFieldType.DATE); + public static final SearchableField LineageIdentifier = new NamedSearchableField("LineageIdentifiers", "lineageIdentifier", "Lineage Identifier", false, SearchableFieldType.STRING); + + public static final SearchableField ContentClaimSection = new NamedSearchableField("ContentClaimSection", "contentClaimSection", "Content Claim Section", false, SearchableFieldType.STRING); + public static final SearchableField ContentClaimContainer = new NamedSearchableField("ContentClaimContainer", "contentClaimContainer", "Content Claim Container", false, SearchableFieldType.STRING); + public static final SearchableField ContentClaimIdentifier = new NamedSearchableField("ContentClaimIdentifier", "contentClaimIdentifier", "Content Claim Identifier", false, SearchableFieldType.STRING); + public static final SearchableField ContentClaimOffset = new NamedSearchableField("ContentClaimOffset", "contentClaimOffset", "Content Claim Offset", false, SearchableFieldType.LONG); + public static final SearchableField SourceQueueIdentifier = new NamedSearchableField("SourceQueueIdentifier", "sourceQueueIdentifier", "Source Queue Identifier", false, SearchableFieldType.STRING); + + private static final Map standardFields; + + static { + final SearchableField[] searchableFields = new SearchableField[]{ + EventTime, FlowFileUUID, Filename, EventType, TransitURI, + ComponentID, AlternateIdentifierURI, FileSize, Relationship, Details, + LineageStartDate, LineageIdentifier, ContentClaimSection, ContentClaimContainer, ContentClaimIdentifier, + ContentClaimOffset, SourceQueueIdentifier}; + + final Map fields = new HashMap<>(); + for (final SearchableField field : searchableFields) { + fields.put(field.getIdentifier(), field); + } + + standardFields = Collections.unmodifiableMap(fields); + } + + private SearchableFields() { + } + + public static Collection getStandardFields() { + return standardFields.values(); + } + + public static SearchableField getSearchableField(final String fieldIdentifier) { + return standardFields.get(fieldIdentifier); + } + + public static SearchableField newSearchableAttribute(final String attributeName) { + return new NamedSearchableField(attributeName, attributeName, attributeName, true); + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java new file mode 100644 index 0000000000..afb56e8bdb --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardLineageResult.java @@ -0,0 +1,324 @@ +/* + * 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.provenance; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.nifi.provenance.lineage.ComputeLineageResult; +import org.apache.nifi.provenance.lineage.EdgeNode; +import org.apache.nifi.provenance.lineage.EventNode; +import org.apache.nifi.provenance.lineage.FlowFileNode; +import org.apache.nifi.provenance.lineage.LineageEdge; +import org.apache.nifi.provenance.lineage.LineageNode; + +/** + * + */ +public class StandardLineageResult implements ComputeLineageResult { + + public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES); + private static final Logger logger = LoggerFactory.getLogger(StandardLineageResult.class); + + private final Collection flowFileUuids; + private final Collection relevantRecords = new ArrayList<>(); + private final Set nodes = new HashSet<>(); + private final Set edges = new HashSet<>(); + private final int numSteps; + private final long creationNanos; + private long computationNanos; + + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + private final Lock writeLock = rwLock.writeLock(); + + private Date expirationDate = null; + private String error = null; + private int numCompletedSteps = 0; + + private volatile boolean canceled = false; + + public StandardLineageResult(final int numSteps, final Collection flowFileUuids) { + this.numSteps = numSteps; + this.creationNanos = System.nanoTime(); + this.flowFileUuids = flowFileUuids; + + updateExpiration(); + } + + @Override + public List getNodes() { + readLock.lock(); + try { + return new ArrayList<>(nodes); + } finally { + readLock.unlock(); + } + } + + @Override + public List getEdges() { + readLock.lock(); + try { + return new ArrayList<>(edges); + } finally { + readLock.unlock(); + } + } + + public int getNumberOfEdges() { + readLock.lock(); + try { + return edges.size(); + } finally { + readLock.unlock(); + } + } + + public int getNumberOfNodes() { + readLock.lock(); + try { + return nodes.size(); + } finally { + readLock.unlock(); + } + } + + public long getComputationTime(final TimeUnit timeUnit) { + readLock.lock(); + try { + return timeUnit.convert(computationNanos, TimeUnit.NANOSECONDS); + } finally { + readLock.unlock(); + } + } + + @Override + public Date getExpiration() { + readLock.lock(); + try { + return expirationDate; + } finally { + readLock.unlock(); + } + } + + @Override + public String getError() { + readLock.lock(); + try { + return error; + } finally { + readLock.unlock(); + } + } + + @Override + public int getPercentComplete() { + readLock.lock(); + try { + return (numSteps < 1) ? 100 : (int) (((float) numCompletedSteps / (float) numSteps) * 100.0F); + } finally { + readLock.unlock(); + } + } + + @Override + public boolean isFinished() { + readLock.lock(); + try { + return numCompletedSteps >= numSteps || canceled; + } finally { + readLock.unlock(); + } + } + + public void setError(final String error) { + writeLock.lock(); + try { + this.error = error; + numCompletedSteps++; + + updateExpiration(); + + if (numCompletedSteps >= numSteps) { + computationNanos = System.nanoTime() - creationNanos; + } + } finally { + writeLock.unlock(); + } + } + + public void update(final Collection records) { + writeLock.lock(); + try { + relevantRecords.addAll(records); + + numCompletedSteps++; + updateExpiration(); + + if (numCompletedSteps >= numSteps && error == null) { + computeLineage(); + computationNanos = System.nanoTime() - creationNanos; + } + } finally { + writeLock.unlock(); + } + } + + /** + * Computes the lineage from the relevant Provenance Event Records. This + * method must be called with the write lock held and is only going to be + * useful after all of the records have been successfully obtained + */ + private void computeLineage() { + final long startNanos = System.nanoTime(); + + nodes.clear(); + edges.clear(); + + Map lastEventMap = new HashMap<>(); // maps FlowFile UUID to last event for that FlowFile + final List sortedRecords = new ArrayList<>(relevantRecords); + Collections.sort(sortedRecords, new Comparator() { + @Override + public int compare(final ProvenanceEventRecord o1, final ProvenanceEventRecord o2) { + // Sort on Event Time, then Event ID. + final int eventTimeComparison = Long.compare(o1.getEventTime(), o2.getEventTime()); + if (eventTimeComparison == 0) { + return Long.compare(o1.getEventId(), o2.getEventId()); + } else { + return eventTimeComparison; + } + } + }); + + // convert the StandardProvenanceRecord objects into Lineage nodes (FlowFileNode, EventNodes). + for (final ProvenanceEventRecord record : sortedRecords) { + final LineageNode lineageNode = new EventNode(record); + final boolean added = nodes.add(lineageNode); + if (!added) { + logger.debug("Did not add {} because it already exists in the 'nodes' set", lineageNode); + } + + // Create an edge that connects this node to the previous node for the same FlowFile UUID. + final LineageNode lastNode = lastEventMap.get(record.getFlowFileUuid()); + if (lastNode != null) { + // We calculate the Edge UUID based on whether or not this event is a SPAWN. + // If this event is a SPAWN, then we want to use the previous node's UUID because a + // SPAWN Event's UUID is not necessarily what we want, since a SPAWN Event's UUID pertains to + // only one of (potentially) many UUIDs associated with the event. Otherwise, we know that + // the UUID of this record is appropriate, so we just use it. + final String edgeUuid; + + switch (record.getEventType()) { + case JOIN: + case CLONE: + case REPLAY: + edgeUuid = lastNode.getFlowFileUuid(); + break; + default: + edgeUuid = record.getFlowFileUuid(); + break; + } + + edges.add(new EdgeNode(edgeUuid, lastNode, lineageNode)); + } + + lastEventMap.put(record.getFlowFileUuid(), lineageNode); + + switch (record.getEventType()) { + case FORK: + case JOIN: + case REPLAY: + case CLONE: { + // For events that create FlowFile nodes, we need to create the FlowFile Nodes and associated Edges, as appropriate + for (final String childUuid : record.getChildUuids()) { + if (flowFileUuids.contains(childUuid)) { + final FlowFileNode childNode = new FlowFileNode(childUuid, record.getEventTime()); + final boolean isNewFlowFile = nodes.add(childNode); + if (!isNewFlowFile) { + final String msg = "Unable to generate Lineage Graph because multiple events were registered claiming to have generated the same FlowFile (UUID = " + childNode.getFlowFileUuid() + ")"; + logger.error(msg); + setError(msg); + return; + } + + edges.add(new EdgeNode(childNode.getFlowFileUuid(), lineageNode, childNode)); + lastEventMap.put(childUuid, childNode); + } + } + for (final String parentUuid : record.getParentUuids()) { + LineageNode lastNodeForParent = lastEventMap.get(parentUuid); + if (lastNodeForParent != null && !lastNodeForParent.equals(lineageNode)) { + edges.add(new EdgeNode(parentUuid, lastNodeForParent, lineageNode)); + } + + lastEventMap.put(parentUuid, lineageNode); + } + } + break; + case RECEIVE: + case CREATE: { + // for a receive event, we want to create a FlowFile Node that represents the FlowFile received + // and create an edge from the Receive Event to the FlowFile Node + final LineageNode flowFileNode = new FlowFileNode(record.getFlowFileUuid(), record.getEventTime()); + final boolean isNewFlowFile = nodes.add(flowFileNode); + if (!isNewFlowFile) { + final String msg = "Found cycle in graph. This indicates that multiple events were registered claiming to have generated the same FlowFile (UUID = " + flowFileNode.getFlowFileUuid() + ")"; + setError(msg); + logger.error(msg); + return; + } + edges.add(new EdgeNode(record.getFlowFileUuid(), lineageNode, flowFileNode)); + lastEventMap.put(record.getFlowFileUuid(), flowFileNode); + } + break; + default: + break; + } + } + + final long nanos = System.nanoTime() - startNanos; + logger.debug("Finished building lineage with {} nodes and {} edges in {} millis", nodes.size(), edges.size(), TimeUnit.NANOSECONDS.toMillis(nanos)); + } + + void cancel() { + this.canceled = true; + } + + /** + * Must be called with write lock! + */ + private void updateExpiration() { + expirationDate = new Date(System.currentTimeMillis() + TTL); + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java new file mode 100644 index 0000000000..cfbae8892c --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardProvenanceEventRecord.java @@ -0,0 +1,752 @@ +/* + * 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.provenance; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.Relationship; + +/** + * Holder for provenance relevant information + *

+ * @author none + */ +public final class StandardProvenanceEventRecord implements ProvenanceEventRecord { + + private final long eventTime; + private final long entryDate; + private final ProvenanceEventType eventType; + private final long lineageStartDate; + private final Set lineageIdentifiers; + private final String componentId; + private final String componentType; + private final String transitUri; + private final String sourceSystemFlowFileIdentifier; + private final String uuid; + private final List parentUuids; + private final List childrenUuids; + private final String alternateIdentifierUri; + private final String details; + private final String relationship; + private final long storageByteOffset; + private final String storageFilename; + private final long eventDuration; + + private final String contentClaimSection; + private final String contentClaimContainer; + private final String contentClaimIdentifier; + private final Long contentClaimOffset; + private final long contentSize; + + private final String previousClaimSection; + private final String previousClaimContainer; + private final String previousClaimIdentifier; + private final Long previousClaimOffset; + private final Long previousSize; + + private final String sourceQueueIdentifier; + + private final Map previousAttributes; + private final Map updatedAttributes; + + private volatile long eventId; + + private StandardProvenanceEventRecord(final Builder builder) { + this.eventTime = builder.eventTime; + this.entryDate = builder.entryDate; + this.eventType = builder.eventType; + this.componentId = builder.componentId; + this.componentType = builder.componentType; + this.transitUri = builder.transitUri; + this.sourceSystemFlowFileIdentifier = builder.sourceSystemFlowFileIdentifier; + this.uuid = builder.uuid; + this.parentUuids = builder.parentUuids; + this.childrenUuids = builder.childrenUuids; + this.alternateIdentifierUri = builder.alternateIdentifierUri; + this.details = builder.details; + this.relationship = builder.relationship; + this.storageByteOffset = builder.storageByteOffset; + this.storageFilename = builder.storageFilename; + this.eventDuration = builder.eventDuration; + this.lineageStartDate = builder.lineageStartDate; + this.lineageIdentifiers = Collections.unmodifiableSet(builder.lineageIdentifiers); + + previousClaimSection = builder.previousClaimSection; + previousClaimContainer = builder.previousClaimContainer; + previousClaimIdentifier = builder.previousClaimIdentifier; + previousClaimOffset = builder.previousClaimOffset; + previousSize = builder.previousSize; + + contentClaimSection = builder.contentClaimSection; + contentClaimContainer = builder.contentClaimContainer; + contentClaimIdentifier = builder.contentClaimIdentifier; + contentClaimOffset = builder.contentClaimOffset; + contentSize = builder.contentSize; + + previousAttributes = builder.previousAttributes == null ? Collections.emptyMap() : Collections.unmodifiableMap(builder.previousAttributes); + updatedAttributes = builder.updatedAttributes == null ? Collections.emptyMap() : Collections.unmodifiableMap(builder.updatedAttributes); + + sourceQueueIdentifier = builder.sourceQueueIdentifier; + + } + + public String getStorageFilename() { + return storageFilename; + } + + public long getStorageByteOffset() { + return storageByteOffset; + } + + void setEventId(final long eventId) { + this.eventId = eventId; + } + + @Override + public long getEventId() { + return eventId; + } + + @Override + public long getEventTime() { + return eventTime; + } + + @Override + public Set getLineageIdentifiers() { + return lineageIdentifiers; + } + + @Override + public long getLineageStartDate() { + return lineageStartDate; + } + + @Override + public long getFileSize() { + return contentSize; + } + + @Override + public Long getPreviousFileSize() { + return previousSize; + } + + @Override + public ProvenanceEventType getEventType() { + return eventType; + } + + @Override + public Map getAttributes() { + final Map allAttrs = new HashMap<>(previousAttributes.size() + updatedAttributes.size()); + allAttrs.putAll(previousAttributes); + for (final Map.Entry entry : updatedAttributes.entrySet()) { + if (entry.getValue() != null) { + allAttrs.put(entry.getKey(), entry.getValue()); + } + } + return allAttrs; + } + + @Override + public String getComponentId() { + return componentId; + } + + @Override + public String getComponentType() { + return componentType; + } + + @Override + public String getTransitUri() { + return transitUri; + } + + @Override + public String getSourceSystemFlowFileIdentifier() { + return sourceSystemFlowFileIdentifier; + } + + @Override + public String getFlowFileUuid() { + return uuid; + } + + @Override + public List getParentUuids() { + return parentUuids == null ? Collections.emptyList() : parentUuids; + } + + @Override + public List getChildUuids() { + return childrenUuids == null ? Collections.emptyList() : childrenUuids; + } + + @Override + public String getAlternateIdentifierUri() { + return alternateIdentifierUri; + } + + @Override + public long getEventDuration() { + return eventDuration; + } + + @Override + public String getDetails() { + return details; + } + + @Override + public String getRelationship() { + return relationship; + } + + @Override + public long getFlowFileEntryDate() { + return entryDate; + } + + @Override + public String getContentClaimSection() { + return contentClaimSection; + } + + @Override + public String getContentClaimContainer() { + return contentClaimContainer; + } + + @Override + public String getContentClaimIdentifier() { + return contentClaimIdentifier; + } + + @Override + public Long getContentClaimOffset() { + return contentClaimOffset; + } + + @Override + public String getSourceQueueIdentifier() { + return sourceQueueIdentifier; + } + + @Override + public Map getPreviousAttributes() { + return previousAttributes; + } + + @Override + public String getPreviousContentClaimContainer() { + return previousClaimContainer; + } + + @Override + public String getPreviousContentClaimIdentifier() { + return previousClaimIdentifier; + } + + @Override + public Long getPreviousContentClaimOffset() { + return previousClaimOffset; + } + + @Override + public String getPreviousContentClaimSection() { + return previousClaimSection; + } + + @Override + public Map getUpdatedAttributes() { + return updatedAttributes; + } + + @Override + public int hashCode() { + final int eventTypeCode; + if (eventType == ProvenanceEventType.CLONE || eventType == ProvenanceEventType.JOIN || eventType == ProvenanceEventType.FORK) { + eventTypeCode = 1472; + } else if (eventType == ProvenanceEventType.REPLAY) { + eventTypeCode = 21479 + (int) (0x7FFFFFFF & eventTime); // use lower bits of event time. + } else { + eventTypeCode = 4812 + eventType.hashCode() + 4 * uuid.hashCode(); + } + + return -37423 + 3 * componentId.hashCode() + (transitUri == null ? 0 : 41 * transitUri.hashCode()) + + (relationship == null ? 0 : 47 * relationship.hashCode()) + 44 * eventTypeCode; + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + if (!(obj instanceof StandardProvenanceEventRecord)) { + return false; + } + + final StandardProvenanceEventRecord other = (StandardProvenanceEventRecord) obj; + // If event ID's are populated and not equal, return false. If they have not yet been populated, do not + // use them in the comparison. + if (eventId > 0L && other.getEventId() > 0L && eventId != other.getEventId()) { + return false; + } + if (eventType != other.eventType) { + return false; + } + + if (!componentId.equals(other.componentId)) { + return false; + } + + if (different(parentUuids, other.parentUuids)) { + return false; + } + + if (different(childrenUuids, other.childrenUuids)) { + return false; + } + + // SPAWN had issues indicating which should be the event's FlowFileUUID in the case that there is 1 parent and 1 child. + if (!uuid.equals(other.uuid)) { + return false; + } + + if (different(transitUri, other.transitUri)) { + return false; + } + + if (different(relationship, other.relationship)) { + return false; + } + + return !(eventType == ProvenanceEventType.REPLAY && eventTime != other.getEventTime()); + } + + private boolean different(final Object a, final Object b) { + if (a == null && b == null) { + return false; + } + if (a == null || b == null) { + return true; + } + + return !a.equals(b); + } + + private boolean different(final List a, final List b) { + if (a == null && b == null) { + return false; + } + + if (a == null && b != null) { + return true; + } + + if (a != null && b == null) { + return true; + } + + if (a.size() != b.size()) { + return true; + } + + final List sortedA = new ArrayList<>(a); + final List sortedB = new ArrayList<>(b); + + Collections.sort(sortedA); + Collections.sort(sortedB); + + for (int i = 0; i < sortedA.size(); i++) { + if (!sortedA.get(i).equals(sortedB.get(i))) { + return true; + } + } + + return false; + } + + @Override + public String toString() { + return "ProvenanceEventRecord [" + + "eventId=" + eventId + + ", eventType=" + eventType + + ", eventTime=" + new Date(eventTime) + + ", uuid=" + uuid + + ", fileSize=" + contentSize + + ", componentId=" + componentId + + ", transitUri=" + transitUri + + ", sourceSystemFlowFileIdentifier=" + sourceSystemFlowFileIdentifier + + ", parentUuids=" + parentUuids + + ", alternateIdentifierUri=" + alternateIdentifierUri + "]"; + } + + public static class Builder implements ProvenanceEventBuilder { + + private long eventTime = System.currentTimeMillis(); + private long entryDate; + private long lineageStartDate; + private Set lineageIdentifiers = new HashSet<>(); + private ProvenanceEventType eventType = null; + private String componentId = null; + private String componentType = null; + private String sourceSystemFlowFileIdentifier = null; + private String transitUri = null; + private String uuid = null; + private List parentUuids = null; + private List childrenUuids = null; + private String contentType = null; + private String alternateIdentifierUri = null; + private String details = null; + private String relationship = null; + private long storageByteOffset = -1L; + private long eventDuration = -1L; + private String storageFilename; + + private String contentClaimSection; + private String contentClaimContainer; + private String contentClaimIdentifier; + private Long contentClaimOffset; + private Long contentSize; + + private String previousClaimSection; + private String previousClaimContainer; + private String previousClaimIdentifier; + private Long previousClaimOffset; + private Long previousSize; + + private String sourceQueueIdentifier; + + private Map previousAttributes; + private Map updatedAttributes; + + @Override + public Builder fromEvent(final ProvenanceEventRecord event) { + eventTime = event.getEventTime(); + entryDate = event.getFlowFileEntryDate(); + lineageStartDate = event.getLineageStartDate(); + lineageIdentifiers = event.getLineageIdentifiers(); + eventType = event.getEventType(); + componentId = event.getComponentId(); + componentType = event.getComponentType(); + transitUri = event.getTransitUri(); + sourceSystemFlowFileIdentifier = event.getSourceSystemFlowFileIdentifier(); + uuid = event.getFlowFileUuid(); + parentUuids = event.getParentUuids(); + childrenUuids = event.getChildUuids(); + alternateIdentifierUri = event.getAlternateIdentifierUri(); + eventDuration = event.getEventDuration(); + previousAttributes = event.getPreviousAttributes(); + updatedAttributes = event.getUpdatedAttributes(); + details = event.getDetails(); + relationship = event.getRelationship(); + + contentClaimSection = event.getContentClaimSection(); + contentClaimContainer = event.getContentClaimContainer(); + contentClaimIdentifier = event.getContentClaimIdentifier(); + contentClaimOffset = event.getContentClaimOffset(); + contentSize = event.getFileSize(); + + previousClaimSection = event.getPreviousContentClaimSection(); + previousClaimContainer = event.getPreviousContentClaimContainer(); + previousClaimIdentifier = event.getPreviousContentClaimIdentifier(); + previousClaimOffset = event.getPreviousContentClaimOffset(); + previousSize = event.getPreviousFileSize(); + + sourceQueueIdentifier = event.getSourceQueueIdentifier(); + + if (event instanceof StandardProvenanceEventRecord) { + final StandardProvenanceEventRecord standardProvEvent = (StandardProvenanceEventRecord) event; + storageByteOffset = standardProvEvent.storageByteOffset; + storageFilename = standardProvEvent.storageFilename; + } + + return this; + } + + @Override + public Builder setFlowFileEntryDate(final long entryDate) { + this.entryDate = entryDate; + return this; + } + + @Override + public Builder setLineageIdentifiers(final Set lineageIdentifiers) { + this.lineageIdentifiers = lineageIdentifiers; + return this; + } + + @Override + public Builder setAttributes(final Map previousAttributes, final Map updatedAttributes) { + this.previousAttributes = previousAttributes; + this.updatedAttributes = updatedAttributes; + return this; + } + + @Override + public Builder setFlowFileUUID(final String uuid) { + this.uuid = uuid; + return this; + } + + public Builder setStorageLocation(final String filename, final long offset) { + this.storageFilename = filename; + this.storageByteOffset = offset; + return this; + } + + @Override + public Builder setEventTime(long eventTime) { + this.eventTime = eventTime; + return this; + } + + @Override + public Builder setEventDuration(final long millis) { + this.eventDuration = millis; + return this; + } + + @Override + public Builder setLineageStartDate(final long startDate) { + this.lineageStartDate = startDate; + return this; + } + + public Builder addLineageIdentifier(final String lineageIdentifier) { + this.lineageIdentifiers.add(lineageIdentifier); + return this; + } + + @Override + public Builder setEventType(ProvenanceEventType eventType) { + this.eventType = eventType; + return this; + } + + @Override + public Builder setComponentId(String componentId) { + this.componentId = componentId; + return this; + } + + @Override + public Builder setComponentType(String componentType) { + this.componentType = componentType; + return this; + } + + @Override + public Builder setSourceSystemFlowFileIdentifier(String sourceSystemFlowFileIdentifier) { + this.sourceSystemFlowFileIdentifier = sourceSystemFlowFileIdentifier; + return this; + } + + @Override + public Builder setTransitUri(String transitUri) { + this.transitUri = transitUri; + return this; + } + + @Override + public Builder addParentFlowFile(final FlowFile parentFlowFile) { + if (this.parentUuids == null) { + this.parentUuids = new ArrayList<>(); + } + this.parentUuids.add(parentFlowFile.getAttribute(CoreAttributes.UUID.key())); + return this; + } + + public Builder addParentUuid(final String uuid) { + if (this.parentUuids == null) { + this.parentUuids = new ArrayList<>(); + } + this.parentUuids.add(uuid); + return this; + } + + @Override + public Builder removeParentFlowFile(final FlowFile parentFlowFile) { + if (this.parentUuids == null) { + return this; + } + + parentUuids.remove(parentFlowFile.getAttribute(CoreAttributes.UUID.key())); + return this; + } + + @Override + public Builder addChildFlowFile(final FlowFile childFlowFile) { + if (this.childrenUuids == null) { + this.childrenUuids = new ArrayList<>(); + } + this.childrenUuids.add(childFlowFile.getAttribute(CoreAttributes.UUID.key())); + return this; + } + + public Builder addChildUuid(final String uuid) { + if (this.childrenUuids == null) { + this.childrenUuids = new ArrayList<>(); + } + this.childrenUuids.add(uuid); + return this; + } + + @Override + public Builder removeChildFlowFile(final FlowFile childFlowFile) { + if (this.childrenUuids == null) { + return this; + } + + childrenUuids.remove(childFlowFile.getAttribute(CoreAttributes.UUID.key())); + return this; + } + + public Builder setContentType(String contentType) { + this.contentType = contentType; + return this; + } + + @Override + public Builder setAlternateIdentifierUri(String alternateIdentifierUri) { + this.alternateIdentifierUri = alternateIdentifierUri; + return this; + } + + @Override + public Builder setDetails(String details) { + this.details = details; + return this; + } + + @Override + public Builder setRelationship(Relationship relationship) { + this.relationship = relationship.getName(); + return this; + } + + public Builder setRelationship(final String relationship) { + this.relationship = relationship; + return this; + } + + @Override + public ProvenanceEventBuilder fromFlowFile(final FlowFile flowFile) { + setFlowFileEntryDate(flowFile.getEntryDate()); + setLineageIdentifiers(flowFile.getLineageIdentifiers()); + setLineageStartDate(flowFile.getLineageStartDate()); + setAttributes(Collections.emptyMap(), flowFile.getAttributes()); + uuid = flowFile.getAttribute(CoreAttributes.UUID.key()); + this.contentSize = flowFile.getSize(); + return this; + } + + @Override + public Builder setPreviousContentClaim(final String container, final String section, final String identifier, final Long offset, final long size) { + previousClaimSection = section; + previousClaimContainer = container; + previousClaimIdentifier = identifier; + previousClaimOffset = offset; + previousSize = size; + return this; + } + + @Override + public Builder setCurrentContentClaim(final String container, final String section, final String identifier, final Long offset, final long size) { + contentClaimSection = section; + contentClaimContainer = container; + contentClaimIdentifier = identifier; + contentClaimOffset = offset; + contentSize = size; + return this; + } + + @Override + public Builder setSourceQueueIdentifier(final String identifier) { + sourceQueueIdentifier = identifier; + return this; + } + + private void assertSet(final Object value, final String name) { + if (value == null) { + throw new IllegalStateException("Cannot create Provenance Event Record because " + name + " is not set"); + } + } + + public ProvenanceEventType getEventType() { + return eventType; + } + + public List getChildUuids() { + return Collections.unmodifiableList(childrenUuids); + } + + public List getParentUuids() { + return Collections.unmodifiableList(parentUuids); + } + + @Override + public StandardProvenanceEventRecord build() { + assertSet(eventType, "Event Type"); + assertSet(componentId, "Component ID"); + assertSet(componentType, "Component Type"); + assertSet(uuid, "FlowFile UUID"); + assertSet(contentSize, "FlowFile Size"); + + switch (eventType) { + case ADDINFO: + if (alternateIdentifierUri == null) { + throw new IllegalStateException("Cannot create Provenance Event Record of type " + eventType + " because no alternate identifiers have been set"); + } + break; + case RECEIVE: + case SEND: + assertSet(transitUri, "Transit URI"); + break; + case ROUTE: + assertSet(relationship, "Relationship"); + break; + case CLONE: + case FORK: + case JOIN: + if ((parentUuids == null || parentUuids.isEmpty()) && (childrenUuids == null || childrenUuids.isEmpty())) { + throw new IllegalStateException("Cannot create Provenance Event Record of type " + eventType + " because no Parent UUIDs or Children UUIDs have been set"); + } + break; + default: + break; + } + + return new StandardProvenanceEventRecord(this); + } + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java new file mode 100644 index 0000000000..9a9a27d79a --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/StandardQueryResult.java @@ -0,0 +1,168 @@ +/* + * 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.provenance; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.search.QueryResult; + +public class StandardQueryResult implements QueryResult { + + public static final int TTL = (int) TimeUnit.MILLISECONDS.convert(30, TimeUnit.MINUTES); + private final Query query; + private final long creationNanos; + + private final int numSteps; + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + + private final Lock writeLock = rwLock.writeLock(); + // guarded by writeLock + private final List matchingRecords = new ArrayList<>(); + private long totalHitCount; + private int numCompletedSteps = 0; + private Date expirationDate; + private String error; + private long queryTime; + + private volatile boolean canceled = false; + + public StandardQueryResult(final Query query, final int numSteps) { + this.query = query; + this.numSteps = numSteps; + this.creationNanos = System.nanoTime(); + + updateExpiration(); + } + + @Override + public List getMatchingEvents() { + readLock.lock(); + try { + if (matchingRecords.size() <= query.getMaxResults()) { + return new ArrayList<>(matchingRecords); + } + + final List copy = new ArrayList<>(query.getMaxResults()); + for (int i = 0; i < query.getMaxResults(); i++) { + copy.add(matchingRecords.get(i)); + } + + return copy; + } finally { + readLock.unlock(); + } + } + + @Override + public long getTotalHitCount() { + readLock.lock(); + try { + return totalHitCount; + } finally { + readLock.unlock(); + } + } + + @Override + public long getQueryTime() { + return queryTime; + } + + @Override + public Date getExpiration() { + return expirationDate; + } + + @Override + public String getError() { + return error; + } + + @Override + public int getPercentComplete() { + readLock.lock(); + try { + return (numSteps < 1) ? 100 : (int) (((float) numCompletedSteps / (float) numSteps) * 100.0F); + } finally { + readLock.unlock(); + } + } + + @Override + public boolean isFinished() { + readLock.lock(); + try { + return numCompletedSteps >= numSteps || canceled; + } finally { + readLock.unlock(); + } + } + + void cancel() { + this.canceled = true; + } + + public void setError(final String error) { + writeLock.lock(); + try { + this.error = error; + numCompletedSteps++; + + updateExpiration(); + if (numCompletedSteps >= numSteps) { + final long searchNanos = System.nanoTime() - creationNanos; + queryTime = TimeUnit.MILLISECONDS.convert(searchNanos, TimeUnit.NANOSECONDS); + } + } finally { + writeLock.unlock(); + } + } + + public void update(final Collection matchingRecords, final long totalHits) { + writeLock.lock(); + try { + this.matchingRecords.addAll(matchingRecords); + this.totalHitCount += totalHits; + + numCompletedSteps++; + updateExpiration(); + + if (numCompletedSteps >= numSteps) { + final long searchNanos = System.nanoTime() - creationNanos; + queryTime = TimeUnit.MILLISECONDS.convert(searchNanos, TimeUnit.NANOSECONDS); + } + } finally { + writeLock.unlock(); + } + } + + /** + * Must be called with write lock! + */ + private void updateExpiration() { + expirationDate = new Date(System.currentTimeMillis() + TTL); + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java new file mode 100644 index 0000000000..0aaf5efb60 --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EdgeNode.java @@ -0,0 +1,74 @@ +/* + * 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.provenance.lineage; + +import static java.util.Objects.requireNonNull; + +public class EdgeNode implements LineageEdge { + + private final String uuid; + private final LineageNode source; + private final LineageNode destination; + + public EdgeNode(final String uuid, final LineageNode source, final LineageNode destination) { + this.uuid = uuid; + this.source = requireNonNull(source); + this.destination = requireNonNull(destination); + } + + @Override + public String getUuid() { + return uuid; + } + + @Override + public LineageNode getSource() { + return source; + } + + @Override + public LineageNode getDestination() { + return destination; + } + + @Override + public int hashCode() { + return 43298293 + source.hashCode() + destination.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + + if (!(obj instanceof EdgeNode)) { + return false; + } + + final EdgeNode other = (EdgeNode) obj; + return (source.equals(other.source) && destination.equals(other.destination)); + } + + @Override + public String toString() { + return "Edge[Source=" + source + ", Destination=" + destination + "]"; + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java new file mode 100644 index 0000000000..12d9a4ff5e --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/EventNode.java @@ -0,0 +1,109 @@ +/* + * 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.provenance.lineage; + +import java.util.List; + +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventType; + +public class EventNode implements ProvenanceEventLineageNode { + + private final ProvenanceEventRecord record; + private String clusterNodeIdentifier = null; + + public EventNode(final ProvenanceEventRecord event) { + this.record = event; + } + + @Override + public String getIdentifier() { + return String.valueOf(getEventIdentifier()); + } + + @Override + public String getClusterNodeIdentifier() { + return clusterNodeIdentifier; + } + + public void setClusterNodeIdentifier(final String nodeIdentifier) { + this.clusterNodeIdentifier = nodeIdentifier; + } + + @Override + public LineageNodeType getNodeType() { + return LineageNodeType.PROVENANCE_EVENT_NODE; + } + + @Override + public ProvenanceEventType getEventType() { + return record.getEventType(); + } + + @Override + public long getTimestamp() { + return record.getEventTime(); + } + + @Override + public long getEventIdentifier() { + return record.getEventId(); + } + + @Override + public String getFlowFileUuid() { + return record.getAttributes().get(CoreAttributes.UUID.key()); + } + + @Override + public List getParentUuids() { + return record.getParentUuids(); + } + + @Override + public List getChildUuids() { + return record.getChildUuids(); + } + + @Override + public int hashCode() { + return 2938472 + record.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (this == obj) { + return true; + } + + if (!(obj instanceof EventNode)) { + return false; + } + + final EventNode other = (EventNode) obj; + return record.equals(other.record); + } + + @Override + public String toString() { + return "Event[ID=" + record.getEventId() + ", Type=" + record.getEventType() + ", UUID=" + record.getFlowFileUuid() + ", Component=" + record.getComponentId() + "]"; + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java new file mode 100644 index 0000000000..c36c38de0e --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileLineage.java @@ -0,0 +1,76 @@ +/* + * 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.provenance.lineage; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import static java.util.Objects.requireNonNull; + +public class FlowFileLineage implements Lineage { + + private final List nodes; + private final List edges; + + public FlowFileLineage(final Collection nodes, final Collection edges) { + this.nodes = new ArrayList<>(requireNonNull(nodes)); + this.edges = new ArrayList<>(requireNonNull(edges)); + } + + @Override + public List getNodes() { + return nodes; + } + + @Override + public List getEdges() { + return edges; + } + + @Override + public int hashCode() { + int sum = 923; + for (final LineageNode node : nodes) { + sum += node.hashCode(); + } + + for (final LineageEdge edge : edges) { + sum += edge.hashCode(); + } + + return sum; + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + + if (obj == this) { + return true; + } + + if (!(obj instanceof FlowFileLineage)) { + return false; + } + + final FlowFileLineage other = (FlowFileLineage) obj; + return nodes.equals(other.nodes) && edges.equals(other.edges); + } +} diff --git a/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java new file mode 100644 index 0000000000..fdc7470d99 --- /dev/null +++ b/commons/data-provenance-utils/src/main/java/org/apache/nifi/provenance/lineage/FlowFileNode.java @@ -0,0 +1,83 @@ +/* + * 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.provenance.lineage; + +import static java.util.Objects.requireNonNull; + +public class FlowFileNode implements LineageNode { + + private final String flowFileUuid; + private final long creationTime; + private String clusterNodeIdentifier; + + public FlowFileNode(final String flowFileUuid, final long flowFileCreationTime) { + this.flowFileUuid = requireNonNull(flowFileUuid); + this.creationTime = flowFileCreationTime; + } + + @Override + public String getIdentifier() { + return flowFileUuid; + } + + @Override + public long getTimestamp() { + return creationTime; + } + + @Override + public String getClusterNodeIdentifier() { + return clusterNodeIdentifier; + } + + @Override + public LineageNodeType getNodeType() { + return LineageNodeType.FLOWFILE_NODE; + } + + @Override + public String getFlowFileUuid() { + return flowFileUuid; + } + + @Override + public int hashCode() { + return 23498723 + flowFileUuid.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + + if (!(obj instanceof FlowFileNode)) { + return false; + } + + final FlowFileNode other = (FlowFileNode) obj; + return flowFileUuid.equals(other.flowFileUuid); + } + + @Override + public String toString() { + return "FlowFile[UUID=" + flowFileUuid + "]"; + } +} diff --git a/commons/flowfile-packager/pom.xml b/commons/flowfile-packager/pom.xml new file mode 100644 index 0000000000..f4001febb3 --- /dev/null +++ b/commons/flowfile-packager/pom.xml @@ -0,0 +1,43 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + flowfile-packager + 0.0.1-SNAPSHOT + jar + + FlowFile Packager + + + + org.apache.commons + commons-compress + 1.9 + + + org.apache.commons + commons-lang3 + 3.3.2 + + + + diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java new file mode 100644 index 0000000000..ae16f999d8 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackager.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; + +public interface FlowFilePackager { + + void packageFlowFile(InputStream in, OutputStream out, Map attributes, long fileSize) throws IOException; + +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java new file mode 100644 index 0000000000..2437279f53 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV1.java @@ -0,0 +1,104 @@ +/* + * 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.util; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Map; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; +import org.apache.commons.lang3.StringEscapeUtils; + +public class FlowFilePackagerV1 implements FlowFilePackager { + + public static final String FILENAME_ATTRIBUTES = "flowfile.attributes"; + public static final String FILENAME_CONTENT = "flowfile.content"; + public static final int DEFAULT_TAR_PERMISSIONS = 0644; + + private final int tarPermissions; + + public FlowFilePackagerV1() { + this(DEFAULT_TAR_PERMISSIONS); + } + + public FlowFilePackagerV1(final int tarPermissions) { + this.tarPermissions = tarPermissions; + } + + @Override + public void packageFlowFile(final InputStream in, final OutputStream out, final Map attributes, final long fileSize) throws IOException { + try (final TarArchiveOutputStream tout = new TarArchiveOutputStream(out)) { + writeAttributesEntry(attributes, tout); + writeContentEntry(tout, in, fileSize); + tout.finish(); + tout.flush(); + tout.close(); + } + } + + private void writeAttributesEntry(final Map attributes, final TarArchiveOutputStream tout) throws IOException { + final StringBuilder sb = new StringBuilder(); + sb.append("\n"); + sb.append(""); + for (final Map.Entry entry : attributes.entrySet()) { + final String escapedKey = StringEscapeUtils.escapeXml(entry.getKey()); + final String escapedValue = StringEscapeUtils.escapeXml(entry.getValue()); + sb.append("\n ").append(escapedValue).append(""); + } + sb.append(""); + + final byte[] metaBytes = sb.toString().getBytes(StandardCharsets.UTF_8); + final TarArchiveEntry attribEntry = new TarArchiveEntry(FILENAME_ATTRIBUTES); + attribEntry.setMode(tarPermissions); + attribEntry.setSize(metaBytes.length); + tout.putArchiveEntry(attribEntry); + tout.write(metaBytes); + tout.closeArchiveEntry(); + } + + private void writeContentEntry(final TarArchiveOutputStream tarOut, final InputStream inStream, final long fileSize) throws IOException { + final TarArchiveEntry entry = new TarArchiveEntry(FILENAME_CONTENT); + entry.setMode(tarPermissions); + entry.setSize(fileSize); + tarOut.putArchiveEntry(entry); + final byte[] buffer = new byte[512 << 10];//512KB + int bytesRead = 0; + while ((bytesRead = inStream.read(buffer)) != -1) { //still more data to read + if (bytesRead > 0) { + tarOut.write(buffer, 0, bytesRead); + } + } + + copy(inStream, tarOut); + tarOut.closeArchiveEntry(); + } + + public static long copy(final InputStream source, final OutputStream destination) throws IOException { + final byte[] buffer = new byte[8192]; + int len; + long totalCount = 0L; + while ((len = source.read(buffer)) > 0) { + destination.write(buffer, 0, len); + totalCount += len; + } + return totalCount; + } + +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java new file mode 100644 index 0000000000..6f9d6b11ce --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV2.java @@ -0,0 +1,146 @@ +/* + * 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.util; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; + +/** + *

+ * Packages a FlowFile, including both its content and its attributes into a + * single file that is stream-friendly. The encoding scheme is as such: + *

+ * + *
+ * Length Field : indicates the number of Flow File Attributes in the stream
+ * 1 to N times (N=number of Flow File Attributes):
+ *      String Field : Flow File Attribute key name
+ *      String Field : Flow File Attribute value
+ * Long : 8 bytes indicating the length of the Flow File content
+ * Content : The next M bytes are the content of the Flow File.
+ * 
+ * + *
+ * Encoding of String Field is as follows:
+ *      Length Field : indicates the length of the String
+ *      1 to N bytes (N=String length, determined by previous field, as described above) : The UTF-8 encoded string value.
+ * 
+ * + *
+ * Encoding of Length Field is as follows:
+ *      First 2 bytes: Indicate length. If both bytes = 255, this is a special value indicating that the length is
+ *                     greater than or equal to 65536 bytes; therefore, the next 4 bytes will indicate the actual length.
+ * 
+ * + *

+ * Note: All byte-order encoding is Network Byte Order (Most Significant Byte + * first) + *

+ * + *

+ * The following example shows the bytes expected if we were to encode a + * FlowFile containing the following attributes where the content is the text + * "Hello World!": + * + *

+ * Attributes: + *

+ * +-------+-------+
+ * | Key   + Value |
+ * + --------------+
+ * | A     | a     |
+ * + --------------+
+ * | B     | b     |
+ * + --------------+
+ * 
Content:
+ * Hello World! + *

+ * Packaged Byte Encoding (In Hexadecimal Form): + *

+ * + *

+ * 00 02 00 01 41 00 01 61
+ * 00 01 42 00 01 62 00 00
+ * 00 00 00 00 00 0C 48 65
+ * 6C 6C 6F 20 57 6F 72 6C
+ * 64 21
+ * 
+ */ +public class FlowFilePackagerV2 implements FlowFilePackager { + + private static final int MAX_VALUE_2_BYTES = 65535; + private final byte[] writeBuffer = new byte[8]; + + @Override + public void packageFlowFile(final InputStream in, final OutputStream out, final Map attributes, final long fileSize) throws IOException { + writeFieldLength(out, attributes.size()); //write out the number of attributes + for (final Map.Entry entry : attributes.entrySet()) { //write out each attribute key/value pair + writeString(entry.getKey(), out); + writeString(entry.getValue(), out); + } + writeLong(out, fileSize);//write out length of data + copy(in, out);//write out the actual flow file payload + } + + private void copy(final InputStream in, final OutputStream out) throws IOException { + final byte[] buffer = new byte[65536]; + int len; + while ((len = in.read(buffer)) > 0) { + out.write(buffer, 0, len); + } + } + + private void writeString(final String val, final OutputStream out) throws IOException { + final byte[] bytes = val.getBytes("UTF-8"); + writeFieldLength(out, bytes.length); + out.write(bytes); + } + + private void writeFieldLength(final OutputStream out, final int numBytes) throws IOException { + // If the value is less than the max value that can be fit into 2 bytes, just use the + // actual value. Otherwise, we will set the first 2 bytes to 255/255 and then use the next + // 4 bytes to indicate the real length. + if (numBytes < MAX_VALUE_2_BYTES) { + writeBuffer[0] = (byte) (numBytes >>> 8); + writeBuffer[1] = (byte) (numBytes); + out.write(writeBuffer, 0, 2); + } else { + writeBuffer[0] = (byte) 0xff; + writeBuffer[1] = (byte) 0xff; + writeBuffer[2] = (byte) (numBytes >>> 24); + writeBuffer[3] = (byte) (numBytes >>> 16); + writeBuffer[4] = (byte) (numBytes >>> 8); + writeBuffer[5] = (byte) (numBytes); + out.write(writeBuffer, 0, 6); + } + } + + private void writeLong(final OutputStream out, final long val) throws IOException { + writeBuffer[0] = (byte) (val >>> 56); + writeBuffer[1] = (byte) (val >>> 48); + writeBuffer[2] = (byte) (val >>> 40); + writeBuffer[3] = (byte) (val >>> 32); + writeBuffer[4] = (byte) (val >>> 24); + writeBuffer[5] = (byte) (val >>> 16); + writeBuffer[6] = (byte) (val >>> 8); + writeBuffer[7] = (byte) (val); + out.write(writeBuffer, 0, 8); + } + +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java new file mode 100644 index 0000000000..181f3e3079 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFilePackagerV3.java @@ -0,0 +1,93 @@ +/* + * 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.util; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; + +public class FlowFilePackagerV3 implements FlowFilePackager { + + public static final byte[] MAGIC_HEADER = {'N', 'i', 'F', 'i', 'F', 'F', '3'}; + private static final int MAX_VALUE_2_BYTES = 65535; + private final byte[] writeBuffer = new byte[8]; + + @Override + public void packageFlowFile(final InputStream in, final OutputStream out, final Map attributes, final long fileSize) throws IOException { + out.write(MAGIC_HEADER); + + if (attributes == null) { + writeFieldLength(out, 0); + } else { + writeFieldLength(out, attributes.size()); //write out the number of attributes + for (final Map.Entry entry : attributes.entrySet()) { //write out each attribute key/value pair + writeString(entry.getKey(), out); + writeString(entry.getValue(), out); + } + } + + writeLong(out, fileSize);//write out length of data + copy(in, out);//write out the actual flow file payload + } + + private void copy(final InputStream in, final OutputStream out) throws IOException { + final byte[] buffer = new byte[65536]; + int len; + while ((len = in.read(buffer)) > 0) { + out.write(buffer, 0, len); + } + } + + private void writeString(final String val, final OutputStream out) throws IOException { + final byte[] bytes = val.getBytes("UTF-8"); + writeFieldLength(out, bytes.length); + out.write(bytes); + } + + private void writeFieldLength(final OutputStream out, final int numBytes) throws IOException { + // If the value is less than the max value that can be fit into 2 bytes, just use the + // actual value. Otherwise, we will set the first 2 bytes to 255/255 and then use the next + // 4 bytes to indicate the real length. + if (numBytes < MAX_VALUE_2_BYTES) { + writeBuffer[0] = (byte) (numBytes >>> 8); + writeBuffer[1] = (byte) (numBytes); + out.write(writeBuffer, 0, 2); + } else { + writeBuffer[0] = (byte) 0xff; + writeBuffer[1] = (byte) 0xff; + writeBuffer[2] = (byte) (numBytes >>> 24); + writeBuffer[3] = (byte) (numBytes >>> 16); + writeBuffer[4] = (byte) (numBytes >>> 8); + writeBuffer[5] = (byte) (numBytes); + out.write(writeBuffer, 0, 6); + } + } + + private void writeLong(final OutputStream out, final long val) throws IOException { + writeBuffer[0] = (byte) (val >>> 56); + writeBuffer[1] = (byte) (val >>> 48); + writeBuffer[2] = (byte) (val >>> 40); + writeBuffer[3] = (byte) (val >>> 32); + writeBuffer[4] = (byte) (val >>> 24); + writeBuffer[5] = (byte) (val >>> 16); + writeBuffer[6] = (byte) (val >>> 8); + writeBuffer[7] = (byte) (val); + out.write(writeBuffer, 0, 8); + } + +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java new file mode 100644 index 0000000000..fd9d92d9b1 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackager.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; + +public interface FlowFileUnpackager { + + Map unpackageFlowFile(InputStream in, OutputStream out) throws IOException; + + boolean hasMoreData() throws IOException; + +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java new file mode 100644 index 0000000000..f8ef3d1ab5 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV1.java @@ -0,0 +1,155 @@ +/* + * 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.util; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.HashMap; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; + +public class FlowFileUnpackagerV1 implements FlowFileUnpackager { + + private int flowFilesRead = 0; + + @Override + public Map unpackageFlowFile(final InputStream in, final OutputStream out) throws IOException { + flowFilesRead++; + final TarArchiveInputStream tarIn = new TarArchiveInputStream(in); + final TarArchiveEntry attribEntry = tarIn.getNextTarEntry(); + if (attribEntry == null) { + return null; + } + + final Map attributes; + if (attribEntry.getName().equals(FlowFilePackagerV1.FILENAME_ATTRIBUTES)) { + attributes = getAttributes(tarIn); + } else { + throw new IOException("Expected two tar entries: " + + FlowFilePackagerV1.FILENAME_CONTENT + " and " + + FlowFilePackagerV1.FILENAME_ATTRIBUTES); + } + + final TarArchiveEntry contentEntry = tarIn.getNextTarEntry(); + + if (contentEntry != null && contentEntry.getName().equals(FlowFilePackagerV1.FILENAME_CONTENT)) { + final byte[] buffer = new byte[512 << 10];//512KB + int bytesRead = 0; + while ((bytesRead = tarIn.read(buffer)) != -1) { //still more data to read + if (bytesRead > 0) { + out.write(buffer, 0, bytesRead); + } + } + out.flush(); + } else { + throw new IOException("Expected two tar entries: " + + FlowFilePackagerV1.FILENAME_CONTENT + " and " + + FlowFilePackagerV1.FILENAME_ATTRIBUTES); + } + + return attributes; + } + + protected Map getAttributes(final TarArchiveInputStream stream) throws IOException { + + final Properties props = new Properties(); + props.loadFromXML(new NonCloseableInputStream(stream)); + + final Map result = new HashMap<>(); + for (final Entry entry : props.entrySet()) { + final Object keyObject = entry.getKey(); + final Object valueObject = entry.getValue(); + if (!(keyObject instanceof String)) { + throw new IOException("Flow file attributes object contains key of type " + + keyObject.getClass().getCanonicalName() + + " but expected java.lang.String"); + } else if (!(keyObject instanceof String)) { + throw new IOException("Flow file attributes object contains value of type " + + keyObject.getClass().getCanonicalName() + + " but expected java.lang.String"); + } + + final String key = (String) keyObject; + final String value = (String) valueObject; + result.put(key, value); + } + + return result; + } + + @Override + public boolean hasMoreData() throws IOException { + return flowFilesRead == 0; + } + + public static final class NonCloseableInputStream extends InputStream { + + final InputStream stream; + + public NonCloseableInputStream(final InputStream stream) { + this.stream = stream; + } + + @Override + public void close() { + } + + @Override + public int read() throws IOException { + return stream.read(); + } + + @Override + public int available() throws IOException { + return stream.available(); + } + + @Override + public synchronized void mark(int readlimit) { + stream.mark(readlimit); + } + + @Override + public synchronized void reset() throws IOException { + stream.reset(); + } + + @Override + public boolean markSupported() { + return stream.markSupported(); + } + + @Override + public long skip(long n) throws IOException { + return stream.skip(n); + } + + @Override + public int read(byte b[], int off, int len) throws IOException { + return stream.read(b, off, len); + } + + @Override + public int read(byte b[]) throws IOException { + return stream.read(b); + } + } +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java new file mode 100644 index 0000000000..500015f746 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV2.java @@ -0,0 +1,143 @@ +/* + * 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.util; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.HashMap; +import java.util.Map; + +public class FlowFileUnpackagerV2 implements FlowFileUnpackager { + + private final byte readBuffer[] = new byte[8192]; + private Map nextAttributes = null; + private boolean haveReadSomething = false; + + @Override + public boolean hasMoreData() throws IOException { + return nextAttributes != null || !haveReadSomething; + } + + protected Map readAttributes(final InputStream in) throws IOException { + final Map attributes = new HashMap<>(); + final Integer numAttributes = readFieldLength(in); //read number of attributes + if (numAttributes == null) { + return null; + } + if (numAttributes == 0) { + throw new IOException("flow files cannot have zero attributes"); + } + for (int i = 0; i < numAttributes; i++) { //read each attribute key/value pair + final String key = readString(in); + final String value = readString(in); + attributes.put(key, value); + } + + return attributes; + } + + @Override + public Map unpackageFlowFile(final InputStream in, final OutputStream out) throws IOException { + final Map attributes; + if (nextAttributes != null) { + attributes = nextAttributes; + } else { + attributes = readAttributes(in); + } + + final long expectedNumBytes = readLong(in); // read length of payload + copy(in, out, expectedNumBytes); // read payload + + nextAttributes = readAttributes(in); + haveReadSomething = true; + + return attributes; + } + + protected String readString(final InputStream in) throws IOException { + final Integer numBytes = readFieldLength(in); + if (numBytes == null) { + throw new EOFException(); + } + final byte[] bytes = new byte[numBytes]; + fillBuffer(in, bytes, numBytes); + return new String(bytes, "UTF-8"); + } + + private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException { + int bytesRead; + int totalBytesRead = 0; + while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) { + totalBytesRead += bytesRead; + } + if (totalBytesRead != length) { + throw new EOFException(); + } + } + + protected long copy(final InputStream in, final OutputStream out, final long numBytes) throws IOException { + int bytesRead; + long totalBytesRead = 0L; + while ((bytesRead = in.read(readBuffer, 0, (int) Math.min(readBuffer.length, numBytes - totalBytesRead))) > 0) { + out.write(readBuffer, 0, bytesRead); + totalBytesRead += bytesRead; + } + + if (totalBytesRead < numBytes) { + throw new EOFException("Expected " + numBytes + " but received " + totalBytesRead); + } + + return totalBytesRead; + } + + protected long readLong(final InputStream in) throws IOException { + fillBuffer(in, readBuffer, 8); + return (((long) readBuffer[0] << 56) + + ((long) (readBuffer[1] & 255) << 48) + + ((long) (readBuffer[2] & 255) << 40) + + ((long) (readBuffer[3] & 255) << 32) + + ((long) (readBuffer[4] & 255) << 24) + + ((readBuffer[5] & 255) << 16) + + ((readBuffer[6] & 255) << 8) + + ((readBuffer[7] & 255))); + } + + private Integer readFieldLength(final InputStream in) throws IOException { + final int firstValue = in.read(); + final int secondValue = in.read(); + if (firstValue < 0) { + return null; + } + if (secondValue < 0) { + throw new EOFException(); + } + if (firstValue == 0xff && secondValue == 0xff) { + int ch1 = in.read(); + int ch2 = in.read(); + int ch3 = in.read(); + int ch4 = in.read(); + if ((ch1 | ch2 | ch3 | ch4) < 0) { + throw new EOFException(); + } + return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4)); + } else { + return ((firstValue << 8) + (secondValue)); + } + } +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java new file mode 100644 index 0000000000..f937585cf3 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/FlowFileUnpackagerV3.java @@ -0,0 +1,161 @@ +/* + * 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.util; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +public class FlowFileUnpackagerV3 implements FlowFileUnpackager { + + private byte[] nextHeader = null; + private boolean haveReadSomething = false; + private final byte readBuffer[] = new byte[8192]; + + @Override + public boolean hasMoreData() throws IOException { + return nextHeader != null || !haveReadSomething; + } + + private byte[] readHeader(final InputStream in) throws IOException { + final byte[] header = new byte[FlowFilePackagerV3.MAGIC_HEADER.length]; + for (int i = 0; i < header.length; i++) { + final int next = in.read(); + if (next < 0) { + if (i == 0) { + return null; + } + + throw new IOException("Not in FlowFile-v3 format"); + } + header[i] = (byte) (next & 0xFF); + } + + return header; + } + + @Override + public Map unpackageFlowFile(final InputStream in, final OutputStream out) throws IOException { + final byte[] header = (nextHeader == null) ? readHeader(in) : nextHeader; + if (!Arrays.equals(header, FlowFilePackagerV3.MAGIC_HEADER)) { + throw new IOException("Not in FlowFile-v3 format"); + } + + final Map attributes = readAttributes(in); + final long expectedNumBytes = readLong(in); // read length of payload + copy(in, out, expectedNumBytes); // read payload + + nextHeader = readHeader(in); + haveReadSomething = true; + + return attributes; + } + + protected Map readAttributes(final InputStream in) throws IOException { + final Map attributes = new HashMap<>(); + final Integer numAttributes = readFieldLength(in); //read number of attributes + if (numAttributes == null) { + return null; + } + if (numAttributes == 0) { + throw new IOException("flow files cannot have zero attributes"); + } + for (int i = 0; i < numAttributes; i++) { //read each attribute key/value pair + final String key = readString(in); + final String value = readString(in); + attributes.put(key, value); + } + + return attributes; + } + + protected String readString(final InputStream in) throws IOException { + final Integer numBytes = readFieldLength(in); + if (numBytes == null) { + throw new EOFException(); + } + final byte[] bytes = new byte[numBytes]; + fillBuffer(in, bytes, numBytes); + return new String(bytes, "UTF-8"); + } + + private void fillBuffer(final InputStream in, final byte[] buffer, final int length) throws IOException { + int bytesRead; + int totalBytesRead = 0; + while ((bytesRead = in.read(buffer, totalBytesRead, length - totalBytesRead)) > 0) { + totalBytesRead += bytesRead; + } + if (totalBytesRead != length) { + throw new EOFException(); + } + } + + protected long copy(final InputStream in, final OutputStream out, final long numBytes) throws IOException { + int bytesRead; + long totalBytesRead = 0L; + while ((bytesRead = in.read(readBuffer, 0, (int) Math.min(readBuffer.length, numBytes - totalBytesRead))) > 0) { + out.write(readBuffer, 0, bytesRead); + totalBytesRead += bytesRead; + } + + if (totalBytesRead < numBytes) { + throw new EOFException("Expected " + numBytes + " but received " + totalBytesRead); + } + + return totalBytesRead; + } + + protected long readLong(final InputStream in) throws IOException { + fillBuffer(in, readBuffer, 8); + return (((long) readBuffer[0] << 56) + + ((long) (readBuffer[1] & 255) << 48) + + ((long) (readBuffer[2] & 255) << 40) + + ((long) (readBuffer[3] & 255) << 32) + + ((long) (readBuffer[4] & 255) << 24) + + ((readBuffer[5] & 255) << 16) + + ((readBuffer[6] & 255) << 8) + + ((readBuffer[7] & 255))); + } + + private Integer readFieldLength(final InputStream in) throws IOException { + final int firstValue = in.read(); + final int secondValue = in.read(); + if (firstValue < 0) { + return null; + } + if (secondValue < 0) { + throw new EOFException(); + } + if (firstValue == 0xff && secondValue == 0xff) { + int ch1 = in.read(); + int ch2 = in.read(); + int ch3 = in.read(); + int ch4 = in.read(); + if ((ch1 | ch2 | ch3 | ch4) < 0) { + throw new EOFException(); + } + return ((ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4)); + } else { + return ((firstValue << 8) + (secondValue)); + } + } + +} diff --git a/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java new file mode 100644 index 0000000000..19f702ca01 --- /dev/null +++ b/commons/flowfile-packager/src/main/java/org/apache/nifi/util/Unpackage.java @@ -0,0 +1,119 @@ +/* + * 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.util; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.Map; +import java.util.UUID; + +public class Unpackage { + + private static void printUsage() { + System.out.println("java " + Unpackage.class.getCanonicalName() + " [ ... ]"); + System.out.println(" : The version of the FlowFile Package format. Valid values are 1, 2, 3"); + System.out.println(" : The FlowFile package to unpack"); + System.out.println(); + } + + public static void main(final String[] args) throws IOException { + if (args.length < 2) { + printUsage(); + return; + } + + final String version = args[0]; + + int inputFileCount = 0; + int outputFileCount = 0; + + for (int i = 1; i < args.length; i++) { + final String filename = args[i]; + final File inFile = new File(filename); + + if (inFile.isDirectory()) { + System.out.println("WARNING: input file " + inFile + " is a directory; skipping"); + continue; + } + + if (!inFile.exists() || !inFile.canRead()) { + System.out.println("ERROR: unable to read file " + inFile); + continue; + } + + final File outputDir = new File(inFile.getAbsolutePath() + ".unpacked"); + if (!outputDir.exists() && !outputDir.mkdir()) { + System.out.println("ERROR: Unable to create directory " + outputDir); + continue; + } + + final File tempFile = new File(outputDir, ".temp." + UUID.randomUUID().toString() + ".unpackage"); + inputFileCount++; + try (final FileInputStream fis = new FileInputStream(inFile); + final BufferedInputStream bufferedIn = new BufferedInputStream(fis)) { + + final FlowFileUnpackager unpackager = createUnpackager(version); + while (unpackager.hasMoreData()) { + outputFileCount++; + final Map attributes; + + try (final FileOutputStream fos = new FileOutputStream(tempFile); + final BufferedOutputStream bufferedOut = new BufferedOutputStream(fos)) { + attributes = unpackager.unpackageFlowFile(bufferedIn, bufferedOut); + } + + String outputFilename = attributes.get("filename"); + if (outputFilename == null) { + outputFilename = attributes.get("nf.file.name"); + } + + final File outputFile = new File(outputDir, outputFilename); + tempFile.renameTo(outputFile); + + final File attributeFilename = new File(outputDir, outputFilename + ".attributes"); + try (final FileOutputStream fos = new FileOutputStream(attributeFilename); + final BufferedOutputStream bufferedOut = new BufferedOutputStream(fos)) { + + for (final Map.Entry entry : attributes.entrySet()) { + bufferedOut.write((entry.getKey() + "=" + entry.getValue() + "\n").getBytes("UTF-8")); + } + } + } + } + } + + System.out.println("Unpacked " + inputFileCount + " packages into " + outputFileCount + " files"); + } + + public static FlowFileUnpackager createUnpackager(final String version) { + switch (version) { + case "1": + return new FlowFileUnpackagerV1(); + case "2": + return new FlowFileUnpackagerV2(); + case "3": + return new FlowFileUnpackagerV3(); + default: + System.out.println("ERROR: Invalid version: " + version + "; must be 1, 2, or 3"); + return null; + } + } +} diff --git a/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java b/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java new file mode 100644 index 0000000000..24cd374645 --- /dev/null +++ b/commons/flowfile-packager/src/test/java/org/apache/nifi/util/TestPackageUnpackageV3.java @@ -0,0 +1,56 @@ +/* + * 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.util; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import org.junit.Test; + +public class TestPackageUnpackageV3 { + + @Test + public void test() throws IOException { + final FlowFilePackager packager = new FlowFilePackagerV3(); + final FlowFileUnpackager unpackager = new FlowFileUnpackagerV3(); + + final byte[] data = "Hello, World!".getBytes("UTF-8"); + final Map map = new HashMap<>(); + map.put("abc", "cba"); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final ByteArrayInputStream in = new ByteArrayInputStream(data); + packager.packageFlowFile(in, baos, map, data.length); + + final byte[] encoded = baos.toByteArray(); + final ByteArrayInputStream encodedIn = new ByteArrayInputStream(encoded); + final ByteArrayOutputStream decodedOut = new ByteArrayOutputStream(); + final Map unpackagedAttributes = unpackager.unpackageFlowFile(encodedIn, decodedOut); + final byte[] decoded = decodedOut.toByteArray(); + + assertEquals(map, unpackagedAttributes); + assertTrue(Arrays.equals(data, decoded)); + } + +} diff --git a/commons/naive-search-ring-buffer/pom.xml b/commons/naive-search-ring-buffer/pom.xml new file mode 100644 index 0000000000..e84be0fdb7 --- /dev/null +++ b/commons/naive-search-ring-buffer/pom.xml @@ -0,0 +1,30 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + naive-search-ring-buffer + 0.0.1-SNAPSHOT + jar + + NiFi Ring Buffer + + diff --git a/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java b/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java new file mode 100644 index 0000000000..85bfd96e67 --- /dev/null +++ b/commons/naive-search-ring-buffer/src/main/java/org/apache/nifi/util/NaiveSearchRingBuffer.java @@ -0,0 +1,135 @@ +/* + * 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.util; + +import java.util.Arrays; + +/** + *

+ * A RingBuffer that can be used to scan byte sequences for subsequences. + *

+ * + *

+ * This class implements an efficient naive search algorithm, which allows the + * user of the library to identify byte sequences in a stream on-the-fly so that + * the stream can be segmented without having to buffer the data. + *

+ * + *

+ * The intended usage paradigm is: + * + *

+ * final byte[] searchSequence = ...;
+ * final CircularBuffer buffer = new CircularBuffer(searchSequence);
+ * while ((int nextByte = in.read()) > 0) {
+ *      if ( buffer.addAndCompare(nextByte) ) {
+ *          // This byte is the last byte in the given sequence
+ *      } else {
+ *          // This byte does not complete the given sequence
+ *      }
+ * }
+ * 
+ * + *

+ */ +public class NaiveSearchRingBuffer { + + private final byte[] lookingFor; + private final int[] buffer; + private int insertionPointer = 0; + private int bufferSize = 0; + + public NaiveSearchRingBuffer(final byte[] lookingFor) { + this.lookingFor = lookingFor; + this.buffer = new int[lookingFor.length]; + Arrays.fill(buffer, -1); + } + + /** + * Returns the contents of the internal buffer, which represents the last X + * bytes added to the buffer, where X is the minimum of the number of bytes + * added to the buffer or the length of the byte sequence for which we are + * looking + * + * @return + */ + public byte[] getBufferContents() { + final int contentLength = Math.min(lookingFor.length, bufferSize); + final byte[] contents = new byte[contentLength]; + for (int i = 0; i < contentLength; i++) { + final byte nextByte = (byte) buffer[(insertionPointer + i) % lookingFor.length]; + contents[i] = nextByte; + } + return contents; + } + + /** + * Returns the oldest byte in the buffer + * + * @return + */ + public int getOldestByte() { + return buffer[insertionPointer]; + } + + /** + * Returns true if the number of bytes that have been added to + * the buffer is at least equal to the length of the byte sequence for which + * we are searching + * + * @return + */ + public boolean isFilled() { + return bufferSize >= buffer.length; + } + + /** + * Clears the internal buffer so that a new search may begin + */ + public void clear() { + Arrays.fill(buffer, -1); + insertionPointer = 0; + bufferSize = 0; + } + + /** + * Add the given byte to the buffer and notify whether or not the byte + * completes the desired byte sequence. + * + * @param data + * @return true if this byte completes the byte sequence, + * false otherwise. + */ + public boolean addAndCompare(final byte data) { + buffer[insertionPointer] = data; + insertionPointer = (insertionPointer + 1) % lookingFor.length; + + bufferSize++; + if (bufferSize < lookingFor.length) { + return false; + } + + for (int i = 0; i < lookingFor.length; i++) { + final byte compare = (byte) buffer[(insertionPointer + i) % lookingFor.length]; + if (compare != lookingFor[i]) { + return false; + } + } + + return true; + } +} diff --git a/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java b/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java new file mode 100644 index 0000000000..0838e968b2 --- /dev/null +++ b/commons/naive-search-ring-buffer/src/test/java/org/apache/nifi/util/TestNaiveSearchRingBuffer.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; + +public class TestNaiveSearchRingBuffer { + + @Test + public void testAddAndCompare() { + final byte[] pattern = new byte[]{ + '\r', '0', 38, 48 + }; + + final byte[] search = new byte[]{ + '\r', '0', 38, 58, 58, 83, 78, '\r', '0', 38, 48, 83, 92, 78, 4, 38 + }; + + final NaiveSearchRingBuffer circ = new NaiveSearchRingBuffer(pattern); + int counter = -1; + for (final byte b : search) { + counter++; + final boolean matched = circ.addAndCompare(b); + if (counter == 10) { + assertTrue(matched); + } else { + assertFalse(matched); + } + } + } + + @Test + public void testGetOldestByte() { + final byte[] pattern = new byte[]{ + '\r', '0', 38, 48 + }; + + final byte[] search = new byte[]{ + '\r', '0', 38, 58, 58, 83, 78, (byte) 223, (byte) 227, (byte) 250, '\r', '0', 38, 48, 83, 92, 78, 4, 38 + }; + + final NaiveSearchRingBuffer circ = new NaiveSearchRingBuffer(pattern); + int counter = -1; + for (final byte b : search) { + counter++; + final boolean matched = circ.addAndCompare(b); + if (counter == 13) { + assertTrue(matched); + } else { + assertFalse(matched); + } + } + } + +} diff --git a/commons/nifi-expression-language/pom.xml b/commons/nifi-expression-language/pom.xml new file mode 100644 index 0000000000..dfb1ea5b03 --- /dev/null +++ b/commons/nifi-expression-language/pom.xml @@ -0,0 +1,59 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-expression-language + 0.0.1-SNAPSHOT + + jar + NiFi Expression Language + + + + + org.antlr + antlr3-maven-plugin + 3.5.2 + + + + antlr + + + + + + + + + + org.antlr + antlr-runtime + 3.5.2 + + + org.apache.nifi + nifi-api + [0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT) + + + diff --git a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g new file mode 100644 index 0000000000..8cb6847f80 --- /dev/null +++ b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g @@ -0,0 +1,207 @@ +/* + * 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. + */ +lexer grammar AttributeExpressionLexer; + +@header { + package org.apache.nifi.attribute.expression.language.antlr; + import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +} + +@rulecatch { + catch(final Exception e) { + throw new AttributeExpressionLanguageParsingException(e); + } +} + +@members { + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new AttributeExpressionLanguageParsingException(sb.toString()); + } + + public void recover(RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new AttributeExpressionLanguageParsingException(sb.toString()); + } +} + + +// PUNCTUATION & SPECIAL CHARACTERS +WHITESPACE : (' '|'\t'|'\n'|'\r')+ { $channel = HIDDEN; }; +COMMENT : '#' ( ~('\n') )* '\n' { $channel = HIDDEN; }; + +DOLLAR : '$'; +LPAREN : '('; +RPAREN : ')'; +LBRACE : '{'; +RBRACE : '}'; +COLON : ':'; +COMMA : ','; +DOT : '.'; +SEMICOLON : ';'; +NUMBER : ('0'..'9')+; + +TRUE : 'true'; +FALSE : 'false'; + +// +// FUNCTION NAMES +// + +// ATTRIBUTE KEY SELECTION FUNCTIONS +ANY_ATTRIBUTE : 'anyAttribute'; +ANY_MATCHING_ATTRIBUTE : 'anyMatchingAttribute'; +ALL_ATTRIBUTES : 'allAttributes'; +ALL_MATCHING_ATTRIBUTES : 'allMatchingAttributes'; +ANY_DELINEATED_VALUE : 'anyDelineatedValue'; +ALL_DELINEATED_VALUES : 'allDelineatedValues'; + +// NO-SUBJECT FUNCTIONS +NEXT_INT : 'nextInt'; +IP : 'ip'; +UUID : 'UUID'; +HOSTNAME : 'hostname'; // requires boolean arg: prefer FQDN +NOW : 'now'; + + +// 0 arg functions +TO_UPPER : 'toUpper'; +TO_LOWER : 'toLower'; +TO_STRING : 'toString'; +LENGTH : 'length'; +TRIM : 'trim'; +IS_NULL : 'isNull'; +NOT_NULL : 'notNull'; +TO_NUMBER : 'toNumber'; +URL_ENCODE : 'urlEncode'; +URL_DECODE : 'urlDecode'; +NOT : 'not'; + +// 1 arg functions +SUBSTRING_AFTER : 'substringAfter'; +SUBSTRING_BEFORE : 'substringBefore'; +SUBSTRING_AFTER_LAST : 'substringAfterLast'; +SUBSTRING_BEFORE_LAST : 'substringBeforeLast'; +STARTS_WITH : 'startsWith'; +ENDS_WITH : 'endsWith'; +CONTAINS : 'contains'; +PREPEND : 'prepend'; +APPEND : 'append'; +INDEX_OF : 'indexOf'; +LAST_INDEX_OF : 'lastIndexOf'; +REPLACE_NULL : 'replaceNull'; +FIND : 'find'; // regex +MATCHES : 'matches'; // regex +EQUALS : 'equals'; +EQUALS_IGNORE_CASE : 'equalsIgnoreCase'; +GREATER_THAN : 'gt'; +LESS_THAN : 'lt'; +GREATER_THAN_OR_EQUAL : 'ge'; +LESS_THAN_OR_EQUAL : 'le'; +FORMAT : 'format'; // takes string date format; uses SimpleDateFormat +TO_DATE : 'toDate'; // takes string date format; converts the subject to a Long based on the date format +MOD : 'mod'; +PLUS : 'plus'; +MINUS : 'minus'; +MULTIPLY : 'multiply'; +DIVIDE : 'divide'; +TO_RADIX : 'toRadix'; +OR : 'or'; +AND : 'and'; + + +// 2 arg functions +SUBSTRING : 'substring'; +REPLACE : 'replace'; +REPLACE_ALL : 'replaceAll'; + + +// STRINGS +STRING_LITERAL +@init{StringBuilder lBuf = new StringBuilder();} + : + ( + '"' + ( + escaped=ESC {lBuf.append(getText());} | + normal = ~( '"' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} + )* + '"' + ) + { + setText(lBuf.toString()); + } + | + ( + '\'' + ( + escaped=ESC {lBuf.append(getText());} | + normal = ~( '\'' | '\\' | '\n' | '\r' | '\t' ) { lBuf.appendCodePoint(normal);} + )* + '\'' + ) + { + setText(lBuf.toString()); + } + ; + + +fragment +ESC + : '\\' + ( + '"' { setText("\""); } + | '\'' { setText("\'"); } + | 'r' { setText("\r"); } + | 'n' { setText("\n"); } + | 't' { setText("\t"); } + | '\\' { setText("\\\\"); } + | nextChar = ~('"' | '\'' | 'r' | 'n' | 't' | '\\') + { + StringBuilder lBuf = new StringBuilder(); lBuf.append("\\\\").appendCodePoint(nextChar); setText(lBuf.toString()); + } + ) + ; + +ATTRIBUTE_NAME : ( + ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n' | '0'..'9') + ~('$' | '{' | '}' | '(' | ')' | '[' | ']' | ',' | ':' | ';' | '/' | '*' | '\'' | ' ' | '\t' | '\r' | '\n')* + ); diff --git a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g new file mode 100644 index 0000000000..cf10fc044e --- /dev/null +++ b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g @@ -0,0 +1,139 @@ +/* + * 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. + */ +parser grammar AttributeExpressionParser; + +options { + output=AST; + tokenVocab=AttributeExpressionLexer; +} + +tokens { + QUERY; + ATTRIBUTE_REFERENCE; + ATTR_NAME; + FUNCTION_CALL; + EXPRESSION; + MULTI_ATTRIBUTE_REFERENCE; + QUOTED_ATTR_NAME; +} + +@header { + package org.apache.nifi.attribute.expression.language.antlr; + import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +} + +@members { + public void displayRecognitionError(String[] tokenNames, RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new AttributeExpressionLanguageParsingException(sb.toString()); + } + + public void recover(final RecognitionException e) { + final StringBuilder sb = new StringBuilder(); + if ( e.token == null ) { + sb.append("Unrecognized token "); + } else { + sb.append("Unexpected token '").append(e.token.getText()).append("' "); + } + sb.append("at line ").append(e.line); + if ( e.approximateLineInfo ) { + sb.append(" (approximately)"); + } + sb.append(", column ").append(e.charPositionInLine); + sb.append(". Query: ").append(e.input.toString()); + + throw new AttributeExpressionLanguageParsingException(sb.toString()); + } +} + +// functions that return Strings +zeroArgString : (TO_UPPER | TO_LOWER | TRIM | TO_STRING | URL_ENCODE | URL_DECODE) LPAREN! RPAREN!; +oneArgString : ((SUBSTRING_BEFORE | SUBSTRING_BEFORE_LAST | SUBSTRING_AFTER | SUBSTRING_AFTER_LAST | REPLACE_NULL | + PREPEND | APPEND | FORMAT | STARTS_WITH | ENDS_WITH | CONTAINS) LPAREN! anyArg RPAREN!) | + (TO_RADIX LPAREN! anyArg (COMMA! anyArg)? RPAREN!); +twoArgString : ((REPLACE | REPLACE_ALL) LPAREN! anyArg COMMA! anyArg RPAREN!) | + (SUBSTRING LPAREN! anyArg (COMMA! anyArg)? RPAREN!); + + +// functions that return Booleans +zeroArgBool : (IS_NULL | NOT_NULL | NOT) LPAREN! RPAREN!; +oneArgBool : ((FIND | MATCHES | EQUALS_IGNORE_CASE) LPAREN! anyArg RPAREN!) | + (GREATER_THAN | LESS_THAN | GREATER_THAN_OR_EQUAL | LESS_THAN_OR_EQUAL) LPAREN! anyArg RPAREN! | + (EQUALS) LPAREN! anyArg RPAREN! | + (AND | OR) LPAREN! anyArg RPAREN!; + + +// functions that return Numbers +zeroArgNum : (LENGTH | TO_NUMBER) LPAREN! RPAREN!; +oneArgNum : ((INDEX_OF | LAST_INDEX_OF) LPAREN! anyArg RPAREN!) | + (TO_DATE LPAREN! anyArg? RPAREN!) | + ((MOD | PLUS | MINUS | MULTIPLY | DIVIDE) LPAREN! anyArg RPAREN!); + +stringFunctionRef : zeroArgString | oneArgString | twoArgString; +booleanFunctionRef : zeroArgBool | oneArgBool; +numberFunctionRef : zeroArgNum | oneArgNum; + +anyArg : NUMBER | numberFunctionRef | STRING_LITERAL | zeroArgString | oneArgString | twoArgString | booleanLiteral | zeroArgBool | oneArgBool | expression; +stringArg : STRING_LITERAL | zeroArgString | oneArgString | twoArgString | expression; +functionRef : stringFunctionRef | booleanFunctionRef | numberFunctionRef; + + + +// Attribute Reference +subject : attrName | expression; +attrName : singleAttrName | multiAttrName; + +singleAttrRef : ATTRIBUTE_NAME | STRING_LITERAL; +singleAttrName : singleAttrRef -> + ^(ATTR_NAME singleAttrRef); + + +multiAttrFunction : ANY_ATTRIBUTE | ANY_MATCHING_ATTRIBUTE | ALL_ATTRIBUTES | ALL_MATCHING_ATTRIBUTES | ANY_DELINEATED_VALUE | ALL_DELINEATED_VALUES; +multiAttrName : multiAttrFunction LPAREN stringArg (COMMA stringArg)* RPAREN -> + ^(MULTI_ATTRIBUTE_REFERENCE multiAttrFunction stringArg*); + +attributeRef : subject -> + ^(ATTRIBUTE_REFERENCE subject); + + +functionCall : functionRef -> + ^(FUNCTION_CALL functionRef); + +booleanLiteral : TRUE | FALSE; +zeroArgStandaloneFunction : (IP | UUID | NOW | NEXT_INT | HOSTNAME) LPAREN! RPAREN!; +oneArgStandaloneFunction : HOSTNAME^ LPAREN! booleanLiteral RPAREN!; +standaloneFunction : zeroArgStandaloneFunction | oneArgStandaloneFunction; + +attributeRefOrFunctionCall : (attributeRef | standaloneFunction); + +expression : DOLLAR LBRACE attributeRefOrFunctionCall (COLON functionCall)* RBRACE -> + ^(EXPRESSION attributeRefOrFunctionCall functionCall*); + +query : expression EOF -> + ^(QUERY expression); diff --git a/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens new file mode 100755 index 0000000000..1b973c8f6c --- /dev/null +++ b/commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/output/AttributeExpressionLexer.tokens @@ -0,0 +1,72 @@ +ALL_ATTRIBUTES=4 +ALL_DELINEATED_VALUES=5 +ALL_MATCHING_ATTRIBUTES=6 +AND=7 +ANY_ATTRIBUTE=8 +ANY_DELINEATED_VALUE=9 +ANY_MATCHING_ATTRIBUTE=10 +APPEND=11 +ATTRIBUTE_NAME=12 +CEIL=13 +COLON=14 +COMMA=15 +CONTAINS=16 +DIVIDE=17 +DOLLAR=18 +DOT=19 +ENDS_WITH=20 +EQUALS=21 +EQUALS_IGNORE_CASE=22 +FALSE=23 +FIND=24 +FLOOR=25 +FORMAT=26 +GREATER_THAN=27 +GREATER_THAN_OR_EQUAL=28 +HOSTNAME=29 +INDEX_OF=30 +IP=31 +IS_NULL=32 +LAST_INDEX_OF=33 +LBRACE=34 +LENGTH=35 +LESS_THAN=36 +LESS_THAN_OR_EQUAL=37 +LPAREN=38 +MATCHES=39 +MINUS=40 +MOD=41 +MULTIPLY=42 +NEXT_INT=43 +NOT=44 +NOT_NULL=45 +NOW=46 +NUMBER=47 +OR=48 +PLUS=49 +PREPEND=50 +RBRACE=51 +REPLACE=52 +REPLACE_ALL=53 +REPLACE_NULL=54 +RPAREN=55 +SEMICOLON=56 +STARTS_WITH=57 +STRING_LITERAL=58 +SUBSTRING=59 +SUBSTRING_AFTER=60 +SUBSTRING_AFTER_LAST=61 +SUBSTRING_BEFORE=62 +SUBSTRING_BEFORE_LAST=63 +TO_DATE=64 +TO_LOWER=65 +TO_NUMBER=66 +TO_RADIX=67 +TO_STRING=68 +TO_UPPER=69 +TRIM=70 +TRUE=71 +URL_DECODE=72 +URL_ENCODE=73 +UUID=74 +WHITESPACE=75 diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java new file mode 100644 index 0000000000..81da47ee31 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language; + +import java.util.Map; + +import org.apache.nifi.expression.AttributeValueDecorator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.exception.ProcessException; + +public class EmptyPreparedQuery implements PreparedQuery { + + private final String value; + + EmptyPreparedQuery(final String value) { + this.value = value; + } + + @Override + public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException { + return value; + } + + @Override + public String evaluateExpressions() throws ProcessException { + return value; + } + + @Override + public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException { + return value; + } + + @Override + public String evaluateExpressions(final FlowFile flowFile) throws ProcessException { + return value; + } + + @Override + public String evaluateExpressions(Map attributes) throws ProcessException { + return value; + } + + @Override + public String evaluateExpressions(Map attributes, AttributeValueDecorator decorator) throws ProcessException { + return value; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java new file mode 100644 index 0000000000..0d1b2c7b79 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language; + +import java.util.Map; + +import org.apache.nifi.expression.AttributeValueDecorator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.exception.ProcessException; + +public interface PreparedQuery { + + String evaluateExpressions(FlowFile flowFile, AttributeValueDecorator decorator) throws ProcessException; + + String evaluateExpressions() throws ProcessException; + + String evaluateExpressions(AttributeValueDecorator decorator) throws ProcessException; + + String evaluateExpressions(FlowFile flowFile) throws ProcessException; + + String evaluateExpressions(Map attributes) throws ProcessException; + + String evaluateExpressions(Map attributes, AttributeValueDecorator decorator) throws ProcessException; + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java new file mode 100644 index 0000000000..6d9cc91ee2 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/Query.java @@ -0,0 +1,1183 @@ +/* + * 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.attribute.expression.language; + +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_ATTRIBUTES; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_DELINEATED_VALUES; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ALL_MATCHING_ATTRIBUTES; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.AND; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_ATTRIBUTE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_DELINEATED_VALUE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ANY_MATCHING_ATTRIBUTE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.APPEND; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTRIBUTE_REFERENCE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ATTR_NAME; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.CONTAINS; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.DIVIDE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.ENDS_WITH; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EQUALS_IGNORE_CASE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.EXPRESSION; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FALSE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FIND; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.FORMAT; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.GREATER_THAN_OR_EQUAL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.HOSTNAME; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.INDEX_OF; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IP; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.IS_NULL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LAST_INDEX_OF; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LENGTH; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.LESS_THAN_OR_EQUAL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MATCHES; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MINUS; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MOD; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTIPLY; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.MULTI_ATTRIBUTE_REFERENCE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NEXT_INT; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOT_NULL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NOW; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.NUMBER; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.OR; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PLUS; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.PREPEND; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_ALL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.REPLACE_NULL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STARTS_WITH; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.STRING_LITERAL; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_DATE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_LOWER; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_NUMBER; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_RADIX; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_STRING; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TO_UPPER; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRIM; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.TRUE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_DECODE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.URL_ENCODE; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.UUID; + +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionLexer; +import org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.cast.BooleanCastEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.cast.DateCastEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.cast.NumberCastEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.cast.StringCastEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.AndEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.AppendEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.AttributeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ContainsEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.DateToNumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.DivideEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.EndsWithEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.EqualsIgnoreCaseEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.FindEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.FormatEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.GreaterThanOrEqualEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.HostnameEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.IPEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.LastIndexOfEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.LengthEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.LessThanOrEqualEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.MatchesEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.MinusEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ModEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.MultiplyEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.NotEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.NotNullEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.NowEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.NumberToDateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.OneUpSequenceEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.OrEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.PlusEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.PrependEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceAllEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ReplaceNullEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.StartsWithEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.StringToDateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringAfterLastEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ToNumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ToUpperEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.TrimEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlDecodeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.UrlEncodeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.UuidEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.literals.BooleanLiteralEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.literals.NumberLiteralEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.AnyAttributeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.DelineatedAttributeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiAttributeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiMatchAttributeEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiNamedAttributeEvaluator; +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException; +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +import org.apache.nifi.expression.AttributeExpression.ResultType; +import org.apache.nifi.expression.AttributeValueDecorator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.exception.ProcessException; + +import org.antlr.runtime.ANTLRStringStream; +import org.antlr.runtime.CharStream; +import org.antlr.runtime.CommonTokenStream; +import org.antlr.runtime.tree.Tree; + +/** + * Class used for creating and evaluating NiFi Expression Language. Once a Query + * has been created, it may be evaluated using the evaluate methods exactly + * once. + */ +public class Query { + + private final String query; + private final Tree tree; + private final Evaluator evaluator; + private final AtomicBoolean evaluated = new AtomicBoolean(false); + + private Query(final String query, final Tree tree, final Evaluator evaluator) { + this.query = query; + this.tree = tree; + this.evaluator = evaluator; + } + + public static boolean isValidExpression(final String value) { + try { + validateExpression(value, false); + return true; + } catch (final ProcessException e) { + return false; + } + } + + public static ResultType getResultType(final String value) throws AttributeExpressionLanguageParsingException { + return Query.compile(value).getResultType(); + } + + public static List extractResultTypes(final String value) throws AttributeExpressionLanguageParsingException { + final List types = new ArrayList<>(); + + for (final Range range : extractExpressionRanges(value)) { + final String text = value.substring(range.getStart(), range.getEnd() + 1); + types.add(getResultType(text)); + } + + return types; + } + + public static List extractExpressions(final String value) throws AttributeExpressionLanguageParsingException { + final List expressions = new ArrayList<>(); + + for (final Range range : extractExpressionRanges(value)) { + expressions.add(value.substring(range.getStart(), range.getEnd() + 1)); + } + + return expressions; + } + + public static List extractExpressionRanges(final String value) throws AttributeExpressionLanguageParsingException { + final List ranges = new ArrayList<>(); + char lastChar = 0; + int embeddedCount = 0; + int expressionStart = -1; + boolean oddDollarCount = false; + int backslashCount = 0; + + charLoop: + for (int i = 0; i < value.length(); i++) { + final char c = value.charAt(i); + + if (expressionStart > -1 && (c == '\'' || c == '"') && (lastChar != '\\' || backslashCount % 2 == 0)) { + final int endQuoteIndex = findEndQuoteChar(value, i); + if (endQuoteIndex < 0) { + break charLoop; + } + + i = endQuoteIndex; + continue; + } + + if (c == '{') { + if (oddDollarCount && lastChar == '$') { + if (embeddedCount == 0) { + expressionStart = i - 1; + } + } + + embeddedCount++; + } else if (c == '}') { + if (embeddedCount <= 0) { + continue; + } + + if (--embeddedCount == 0) { + if (expressionStart > -1) { + // ended expression. Add a new range. + final Range range = new Range(expressionStart, i); + ranges.add(range); + } + + expressionStart = -1; + } + } else if (c == '$') { + oddDollarCount = !oddDollarCount; + } else if (c == '\\') { + backslashCount++; + } else { + oddDollarCount = false; + } + + lastChar = c; + } + + return ranges; + } + + /** + * + * + * @param value + * @param allowSurroundingCharacters + * @throws AttributeExpressionLanguageParsingException + */ + public static void validateExpression(final String value, final boolean allowSurroundingCharacters) throws AttributeExpressionLanguageParsingException { + if (!allowSurroundingCharacters) { + final List ranges = extractExpressionRanges(value); + if (ranges.size() > 1) { + throw new AttributeExpressionLanguageParsingException("Found multiple Expressions but expected only 1"); + } + + if (ranges.isEmpty()) { + throw new AttributeExpressionLanguageParsingException("No Expressions found"); + } + + final Range range = ranges.get(0); + final String expression = value.substring(range.getStart(), range.getEnd() + 1); + Query.compile(expression); + + if (range.getStart() > 0 || range.getEnd() < value.length() - 1) { + throw new AttributeExpressionLanguageParsingException("Found characters outside of Expression"); + } + } else { + for (final Range range : extractExpressionRanges(value)) { + final String expression = value.substring(range.getStart(), range.getEnd() + 1); + Query.compile(expression); + } + } + } + + static int findEndQuoteChar(final String value, final int quoteStart) { + final char quoteChar = value.charAt(quoteStart); + + int backslashCount = 0; + char lastChar = 0; + for (int i = quoteStart + 1; i < value.length(); i++) { + final char c = value.charAt(i); + + if (c == '\\') { + backslashCount++; + } else if (c == quoteChar && ((backslashCount % 2 == 0) || lastChar != '\\')) { + return i; + } + + lastChar = c; + } + + return -1; + } + + static String evaluateExpression(final Tree tree, final String queryText, final Map expressionMap, final AttributeValueDecorator decorator) throws ProcessException { + final Object evaluated = Query.fromTree(tree, queryText).evaluate(expressionMap).getValue(); + if (evaluated == null) { + return null; + } + + final String value = evaluated.toString(); + final String escaped = value.replace("$$", "$"); + return (decorator == null) ? escaped : decorator.decorate(escaped); + } + + static String evaluateExpressions(final String rawValue, Map expressionMap) throws ProcessException { + return evaluateExpressions(rawValue, expressionMap, null); + } + + static String evaluateExpressions(final String rawValue) throws ProcessException { + return evaluateExpressions(rawValue, createExpressionMap(null), null); + } + + static String evaluateExpressions(final String rawValue, final FlowFile flowFile) throws ProcessException { + return evaluateExpressions(rawValue, createExpressionMap(flowFile), null); + } + + static String evaluateExpressions(final String rawValue, Map expressionMap, final AttributeValueDecorator decorator) throws ProcessException { + return Query.prepare(rawValue).evaluateExpressions(expressionMap, decorator); + } + + public static String evaluateExpressions(final String rawValue, final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException { + if (rawValue == null) { + return null; + } + + final Map expressionMap = createExpressionMap(flowFile); + return evaluateExpressions(rawValue, expressionMap, decorator); + } + + private static Evaluator getRootSubjectEvaluator(final Evaluator evaluator) { + if (evaluator == null) { + return null; + } + + final Evaluator subject = evaluator.getSubjectEvaluator(); + if (subject == null) { + return evaluator; + } + + return getRootSubjectEvaluator(subject); + } + + /** + * Un-escapes ${...} patterns that were escaped + * + * @param value + * @return + */ + public static String unescape(final String value) { + return value.replaceAll("\\$\\$(?=\\$*\\{.*?\\})", "\\$"); + } + + static Map createExpressionMap(final FlowFile flowFile) { + final Map attributeMap = flowFile == null ? new HashMap() : flowFile.getAttributes(); + final Map envMap = System.getenv(); + final Map sysProps = System.getProperties(); + + final Map flowFileProps = new HashMap<>(); + if (flowFile != null) { + flowFileProps.put("flowFileId", String.valueOf(flowFile.getId())); + flowFileProps.put("fileSize", String.valueOf(flowFile.getSize())); + flowFileProps.put("entryDate", String.valueOf(flowFile.getEntryDate())); + flowFileProps.put("lineageStartDate", String.valueOf(flowFile.getLineageStartDate())); + } + + return wrap(attributeMap, flowFileProps, envMap, sysProps); + } + + private static Map wrap(final Map attributes, final Map flowFileProps, + final Map env, final Map sysProps) { + @SuppressWarnings("rawtypes") + final Map[] maps = new Map[]{attributes, flowFileProps, env, sysProps}; + + return new Map() { + @Override + public int size() { + int size = 0; + for (final Map map : maps) { + size += map.size(); + } + return size; + } + + @Override + public boolean isEmpty() { + for (final Map map : maps) { + if (!map.isEmpty()) { + return false; + } + } + return true; + } + + @Override + public boolean containsKey(final Object key) { + if (key == null) { + return false; + } + if (!(key instanceof String)) { + return false; + } + + for (final Map map : maps) { + if (map.containsKey(key)) { + return true; + } + } + return false; + } + + @Override + public boolean containsValue(final Object value) { + for (final Map map : maps) { + if (map.containsValue(value)) { + return true; + } + } + return false; + } + + @Override + @SuppressWarnings("rawtypes") + public String get(final Object key) { + if (key == null) { + throw new IllegalArgumentException("Null Keys are not allowed"); + } + if (!(key instanceof String)) { + return null; + } + + for (final Map map : maps) { + final Object val = map.get(key); + if (val != null) { + return String.valueOf(val); + } + } + return null; + } + + @Override + public String put(String key, String value) { + throw new UnsupportedOperationException(); + } + + @Override + public String remove(final Object key) { + throw new UnsupportedOperationException(); + } + + @Override + public void putAll(final Map m) { + throw new UnsupportedOperationException(); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(); + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public Set keySet() { + final Set keySet = new HashSet<>(); + for (final Map map : maps) { + keySet.addAll(map.keySet()); + } + return keySet; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public Collection values() { + final Set values = new HashSet<>(); + for (final Map map : maps) { + values.addAll(map.values()); + } + return values; + } + + @Override + @SuppressWarnings({"unchecked", "rawtypes"}) + public Set> entrySet() { + final Set> entrySet = new HashSet<>(); + for (final Map map : maps) { + entrySet.addAll(map.entrySet()); + } + return entrySet; + } + + }; + } + + public static Query fromTree(final Tree tree, final String text) { + return new Query(text, tree, buildEvaluator(tree)); + } + + public static Tree compileTree(final String query) throws AttributeExpressionLanguageParsingException { + try { + final CommonTokenStream lexerTokenStream = createTokenStream(query); + final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream); + final Tree ast = (Tree) parser.query().getTree(); + final Tree tree = ast.getChild(0); + + // ensure that we are able to build the evaluators, so that we validate syntax + buildEvaluator(tree); + return tree; + } catch (final AttributeExpressionLanguageParsingException e) { + throw e; + } catch (final Exception e) { + throw new AttributeExpressionLanguageParsingException(e); + } + } + + public static PreparedQuery prepare(final String query) throws AttributeExpressionLanguageParsingException { + if (query == null) { + return new EmptyPreparedQuery(null); + } + + final List ranges = extractExpressionRanges(query); + + if (ranges.isEmpty()) { + return new EmptyPreparedQuery(query.replace("$$", "$")); + } + + final List substrings = new ArrayList<>(); + final Map trees = new HashMap<>(); + + int lastIndex = 0; + for (final Range range : ranges) { + if (range.getStart() > lastIndex) { + substrings.add(query.substring(lastIndex, range.getStart()).replace("$$", "$")); + lastIndex = range.getEnd() + 1; + } + + final String treeText = query.substring(range.getStart(), range.getEnd() + 1).replace("$$", "$"); + substrings.add(treeText); + trees.put(treeText, Query.compileTree(treeText)); + lastIndex = range.getEnd() + 1; + } + + final Range lastRange = ranges.get(ranges.size() - 1); + if (lastRange.getEnd() + 1 < query.length()) { + final String treeText = query.substring(lastRange.getEnd() + 1).replace("$$", "$"); + substrings.add(treeText); + } + + return new StandardPreparedQuery(substrings, trees); + } + + public static Query compile(final String query) throws AttributeExpressionLanguageParsingException { + try { + final CommonTokenStream lexerTokenStream = createTokenStream(query); + final AttributeExpressionParser parser = new AttributeExpressionParser(lexerTokenStream); + final Tree ast = (Tree) parser.query().getTree(); + final Tree tree = ast.getChild(0); + + return new Query(query, tree, buildEvaluator(tree)); + } catch (final AttributeExpressionLanguageParsingException e) { + throw e; + } catch (final Exception e) { + throw new AttributeExpressionLanguageParsingException(e); + } + } + + private static CommonTokenStream createTokenStream(final String expression) throws AttributeExpressionLanguageParsingException { + final CharStream input = new ANTLRStringStream(expression); + final AttributeExpressionLexer lexer = new AttributeExpressionLexer(input); + return new CommonTokenStream(lexer); + } + + public ResultType getResultType() { + return evaluator.getResultType(); + } + + QueryResult evaluate() { + return evaluate(createExpressionMap(null)); + } + + QueryResult evaluate(final FlowFile flowFile) { + return evaluate(createExpressionMap(flowFile)); + } + + QueryResult evaluate(final Map attributes) { + if (evaluated.getAndSet(true)) { + throw new IllegalStateException("A Query cannot be evaluated more than once"); + } + + Evaluator chosenEvaluator = evaluator; + final Evaluator rootEvaluator = getRootSubjectEvaluator(evaluator); + if (rootEvaluator != null) { + if (rootEvaluator instanceof MultiAttributeEvaluator) { + if (evaluator.getResultType() != ResultType.BOOLEAN) { + throw new AttributeExpressionLanguageParsingException("Found Multi-Attribute function but return type is " + evaluator.getResultType() + ", not " + ResultType.BOOLEAN + ", for query: " + query); + } + + final MultiAttributeEvaluator multiAttrEval = (MultiAttributeEvaluator) rootEvaluator; + + switch (multiAttrEval.getEvaluationType()) { + case ANY_ATTRIBUTE: + case ANY_MATCHING_ATTRIBUTE: + case ANY_DELINEATED_VALUE: + chosenEvaluator = new AnyAttributeEvaluator((BooleanEvaluator) evaluator, multiAttrEval); + break; + case ALL_ATTRIBUTES: + case ALL_MATCHING_ATTRIBUTES: + case ALL_DELINEATED_VALUES: + chosenEvaluator = new AllAttributesEvaluator((BooleanEvaluator) evaluator, multiAttrEval); + break; + } + } + } + + return chosenEvaluator.evaluate(attributes); + } + + Tree getTree() { + return this.tree; + } + + @Override + public String toString() { + return "Query [" + query + "]"; + } + + private static StringEvaluator newStringLiteralEvaluator(final String literalValue) { + if (literalValue == null || literalValue.length() < 2) { + return new StringLiteralEvaluator(literalValue); + } + + final List ranges = extractExpressionRanges(literalValue); + if (ranges.isEmpty()) { + return new StringLiteralEvaluator(literalValue); + } + + final List> evaluators = new ArrayList<>(); + + int lastIndex = 0; + for (final Range range : ranges) { + if (range.getStart() > lastIndex) { + evaluators.add(newStringLiteralEvaluator(literalValue.substring(lastIndex, range.getStart()))); + } + + final String treeText = literalValue.substring(range.getStart(), range.getEnd() + 1); + evaluators.add(buildEvaluator(compileTree(treeText))); + lastIndex = range.getEnd() + 1; + } + + final Range lastRange = ranges.get(ranges.size() - 1); + if (lastRange.getEnd() + 1 < literalValue.length()) { + final String treeText = literalValue.substring(lastRange.getEnd() + 1); + evaluators.add(newStringLiteralEvaluator(treeText)); + } + + if (evaluators.size() == 1) { + return toStringEvaluator(evaluators.get(0)); + } + + StringEvaluator lastEvaluator = toStringEvaluator(evaluators.get(0)); + for (int i = 1; i < evaluators.size(); i++) { + lastEvaluator = new AppendEvaluator(lastEvaluator, toStringEvaluator(evaluators.get(i))); + } + + return lastEvaluator; + } + + private static Evaluator buildEvaluator(final Tree tree) { + switch (tree.getType()) { + case EXPRESSION: { + return buildExpressionEvaluator(tree); + } + case ATTRIBUTE_REFERENCE: { + final Evaluator childEvaluator = buildEvaluator(tree.getChild(0)); + if (childEvaluator instanceof MultiAttributeEvaluator) { + return childEvaluator; + } + return new AttributeEvaluator(toStringEvaluator(childEvaluator)); + } + case MULTI_ATTRIBUTE_REFERENCE: { + + final Tree functionTypeTree = tree.getChild(0); + final int multiAttrType = functionTypeTree.getType(); + if (multiAttrType == ANY_DELINEATED_VALUE || multiAttrType == ALL_DELINEATED_VALUES) { + final StringEvaluator delineatedValueEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(1))); + final StringEvaluator delimiterEvaluator = toStringEvaluator(buildEvaluator(tree.getChild(2))); + + return new DelineatedAttributeEvaluator(delineatedValueEvaluator, delimiterEvaluator, multiAttrType); + } + + final List attributeNames = new ArrayList<>(); + for (int i = 1; i < tree.getChildCount(); i++) { // skip the first child because that's the name of the multi-attribute function + attributeNames.add(newStringLiteralEvaluator(tree.getChild(i).getText()).evaluate(null).getValue()); + } + + switch (multiAttrType) { + case ALL_ATTRIBUTES: + for (final String attributeName : attributeNames) { + try { + FlowFile.KeyValidator.validateKey(attributeName); + } catch (final IllegalArgumentException iae) { + throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage()); + } + } + + return new MultiNamedAttributeEvaluator(attributeNames, ALL_ATTRIBUTES); + case ALL_MATCHING_ATTRIBUTES: + return new MultiMatchAttributeEvaluator(attributeNames, ALL_MATCHING_ATTRIBUTES); + case ANY_ATTRIBUTE: + for (final String attributeName : attributeNames) { + try { + FlowFile.KeyValidator.validateKey(attributeName); + } catch (final IllegalArgumentException iae) { + throw new AttributeExpressionLanguageParsingException("Invalid Attribute Name: " + attributeName + ". " + iae.getMessage()); + } + } + + return new MultiNamedAttributeEvaluator(attributeNames, ANY_ATTRIBUTE); + case ANY_MATCHING_ATTRIBUTE: + return new MultiMatchAttributeEvaluator(attributeNames, ANY_MATCHING_ATTRIBUTE); + default: + throw new AssertionError("Illegal Multi-Attribute Reference: " + functionTypeTree.toString()); + } + } + case ATTR_NAME: { + return newStringLiteralEvaluator(tree.getChild(0).getText()); + } + case NUMBER: { + return new NumberLiteralEvaluator(tree.getText()); + } + case STRING_LITERAL: { + return newStringLiteralEvaluator(tree.getText()); + } + case TRUE: + case FALSE: + return buildBooleanEvaluator(tree); + case UUID: { + return new UuidEvaluator(); + } + case NOW: { + return new NowEvaluator(); + } + case IP: { + try { + return new IPEvaluator(); + } catch (final UnknownHostException e) { + throw new AttributeExpressionLanguageException(e); + } + } + case HOSTNAME: { + if (tree.getChildCount() == 0) { + try { + return new HostnameEvaluator(false); + } catch (UnknownHostException e) { + throw new AttributeExpressionLanguageException(e); + } + } else if (tree.getChildCount() == 1) { + final Tree childTree = tree.getChild(0); + try { + switch (childTree.getType()) { + case TRUE: + return new HostnameEvaluator(true); + case FALSE: + return new HostnameEvaluator(false); + default: + throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter"); + } + } catch (UnknownHostException e) { + throw new AttributeExpressionLanguageException(e); + } + } else { + throw new AttributeExpressionLanguageParsingException("Call to hostname() must take 0 or 1 (boolean) parameter"); + } + } + case NEXT_INT: { + return new OneUpSequenceEvaluator(); + } + default: + throw new AttributeExpressionLanguageParsingException("Unexpected token: " + tree.toString()); + } + } + + private static Evaluator buildBooleanEvaluator(final Tree tree) { + switch (tree.getType()) { + case TRUE: + return new BooleanLiteralEvaluator(true); + case FALSE: + return new BooleanLiteralEvaluator(false); + } + throw new AttributeExpressionLanguageParsingException("Cannot build Boolean evaluator from tree " + tree.toString()); + } + + private static Evaluator buildExpressionEvaluator(final Tree tree) { + if (tree.getChildCount() == 0) { + throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children"); + } + if (tree.getChildCount() == 1) { + return buildEvaluator(tree.getChild(0)); + } else { + // we can chain together functions in the form of: + // ${x:trim():substring(1,2):trim()} + // in this case, the subject of the right-most function is the function to its left; its + // subject is the function to its left (the first trim()), and its subject is the value of + // the 'x' attribute. We accomplish this logic by iterating over all of the children of the + // tree from the right-most child going left-ward. + return buildFunctionExpressionEvaluator(tree, 0); + } + } + + private static Evaluator buildFunctionExpressionEvaluator(final Tree tree, final int offset) { + if (tree.getChildCount() == 0) { + throw new AttributeExpressionLanguageParsingException("EXPRESSION tree node has no children"); + } + final int firstChildIndex = tree.getChildCount() - offset - 1; + if (firstChildIndex == 0) { + return buildEvaluator(tree.getChild(0)); + } + + final Tree functionTree = tree.getChild(firstChildIndex); + final Evaluator subjectEvaluator = buildFunctionExpressionEvaluator(tree, offset + 1); + + final Tree functionNameTree = functionTree.getChild(0); + final List> argEvaluators = new ArrayList<>(); + for (int i = 1; i < functionTree.getChildCount(); i++) { + argEvaluators.add(buildEvaluator(functionTree.getChild(i))); + } + return buildFunctionEvaluator(functionNameTree, subjectEvaluator, argEvaluators); + } + + private static List> verifyArgCount(final List> args, final int count, final String functionName) { + if (args.size() != count) { + throw new AttributeExpressionLanguageParsingException(functionName + "() function takes " + count + " arguments"); + } + return args; + } + + private static StringEvaluator toStringEvaluator(final Evaluator evaluator) { + return toStringEvaluator(evaluator, null); + } + + private static StringEvaluator toStringEvaluator(final Evaluator evaluator, final String location) { + if (evaluator.getResultType() == ResultType.STRING) { + return (StringEvaluator) evaluator; + } + + return new StringCastEvaluator(evaluator); + } + + private static BooleanEvaluator toBooleanEvaluator(final Evaluator evaluator, final String location) { + switch (evaluator.getResultType()) { + case BOOLEAN: + return (BooleanEvaluator) evaluator; + case STRING: + return new BooleanCastEvaluator((StringEvaluator) evaluator); + default: + throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.BOOLEAN + + (location == null ? "" : " at location [" + location + "]")); + } + + } + + private static BooleanEvaluator toBooleanEvaluator(final Evaluator evaluator) { + return toBooleanEvaluator(evaluator, null); + } + + private static NumberEvaluator toNumberEvaluator(final Evaluator evaluator) { + return toNumberEvaluator(evaluator, null); + } + + private static NumberEvaluator toNumberEvaluator(final Evaluator evaluator, final String location) { + switch (evaluator.getResultType()) { + case NUMBER: + return (NumberEvaluator) evaluator; + case STRING: + return new NumberCastEvaluator((StringEvaluator) evaluator); + default: + throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + evaluator.getResultType() + " to " + ResultType.NUMBER + + (location == null ? "" : " at location [" + location + "]")); + } + } + + private static DateEvaluator toDateEvaluator(final Evaluator evaluator) { + return toDateEvaluator(evaluator, null); + } + + private static DateEvaluator toDateEvaluator(final Evaluator evaluator, final String location) { + if (evaluator.getResultType() == ResultType.DATE) { + return (DateEvaluator) evaluator; + } + + return new DateCastEvaluator(evaluator); + } + + private static Evaluator buildFunctionEvaluator(final Tree tree, final Evaluator subjectEvaluator, final List> argEvaluators) { + switch (tree.getType()) { + case TRIM: { + verifyArgCount(argEvaluators, 0, "trim"); + return new TrimEvaluator(toStringEvaluator(subjectEvaluator)); + } + case TO_STRING: { + verifyArgCount(argEvaluators, 0, "toString"); + return new ToStringEvaluator(subjectEvaluator); + } + case TO_LOWER: { + verifyArgCount(argEvaluators, 0, "toLower"); + return new ToLowerEvaluator(toStringEvaluator(subjectEvaluator)); + } + case TO_UPPER: { + verifyArgCount(argEvaluators, 0, "toUpper"); + return new ToUpperEvaluator(toStringEvaluator(subjectEvaluator)); + } + case URL_ENCODE: { + verifyArgCount(argEvaluators, 0, "urlEncode"); + return new UrlEncodeEvaluator(toStringEvaluator(subjectEvaluator)); + } + case URL_DECODE: { + verifyArgCount(argEvaluators, 0, "urlDecode"); + return new UrlDecodeEvaluator(toStringEvaluator(subjectEvaluator)); + } + case SUBSTRING_BEFORE: { + verifyArgCount(argEvaluators, 1, "substringBefore"); + return new SubstringBeforeEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringBefore")); + } + case SUBSTRING_BEFORE_LAST: { + verifyArgCount(argEvaluators, 1, "substringBeforeLast"); + return new SubstringBeforeLastEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringBeforeLast")); + } + case SUBSTRING_AFTER: { + verifyArgCount(argEvaluators, 1, "substringAfter"); + return new SubstringAfterEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringAfter")); + } + case SUBSTRING_AFTER_LAST: { + verifyArgCount(argEvaluators, 1, "substringAfterLast"); + return new SubstringAfterLastEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to substringAfterLast")); + } + case REPLACE_NULL: { + verifyArgCount(argEvaluators, 1, "replaceNull"); + return new ReplaceNullEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to replaceNull")); + } + case REPLACE: { + verifyArgCount(argEvaluators, 2, "replace"); + return new ReplaceEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to replace"), + toStringEvaluator(argEvaluators.get(1), "second argument to replace")); + } + case REPLACE_ALL: { + verifyArgCount(argEvaluators, 2, "replaceAll"); + return new ReplaceAllEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to replaceAll"), + toStringEvaluator(argEvaluators.get(1), "second argument to replaceAll")); + } + case APPEND: { + verifyArgCount(argEvaluators, 1, "append"); + return new AppendEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to append")); + } + case PREPEND: { + verifyArgCount(argEvaluators, 1, "prepend"); + return new PrependEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to prepend")); + } + case SUBSTRING: { + final int numArgs = argEvaluators.size(); + if (numArgs == 1) { + return new SubstringEvaluator(toStringEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to substring")); + } else if (numArgs == 2) { + return new SubstringEvaluator(toStringEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to substring"), + toNumberEvaluator(argEvaluators.get(1), "second argument to substring")); + } else { + throw new AttributeExpressionLanguageParsingException("substring() function can take either 1 or 2 arguments but cannot take " + numArgs + " arguments"); + } + } + case IS_NULL: { + verifyArgCount(argEvaluators, 0, "isNull"); + return new IsNullEvaluator(toStringEvaluator(subjectEvaluator)); + } + case NOT_NULL: { + verifyArgCount(argEvaluators, 0, "notNull"); + return new NotNullEvaluator(toStringEvaluator(subjectEvaluator)); + } + case STARTS_WITH: { + verifyArgCount(argEvaluators, 1, "startsWith"); + return new StartsWithEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to startsWith")); + } + case ENDS_WITH: { + verifyArgCount(argEvaluators, 1, "endsWith"); + return new EndsWithEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to endsWith")); + } + case CONTAINS: { + verifyArgCount(argEvaluators, 1, "contains"); + return new ContainsEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to contains")); + } + case FIND: { + verifyArgCount(argEvaluators, 1, "find"); + return new FindEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to find")); + } + case MATCHES: { + verifyArgCount(argEvaluators, 1, "matches"); + return new MatchesEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to matches")); + } + case EQUALS: { + verifyArgCount(argEvaluators, 1, "equals"); + return new EqualsEvaluator(subjectEvaluator, argEvaluators.get(0)); + } + case EQUALS_IGNORE_CASE: { + verifyArgCount(argEvaluators, 1, "equalsIgnoreCase"); + return new EqualsIgnoreCaseEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to equalsIgnoreCase")); + } + case GREATER_THAN: { + verifyArgCount(argEvaluators, 1, "gt"); + return new GreaterThanEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to gt")); + } + case GREATER_THAN_OR_EQUAL: { + verifyArgCount(argEvaluators, 1, "ge"); + return new GreaterThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to ge")); + } + case LESS_THAN: { + verifyArgCount(argEvaluators, 1, "lt"); + return new LessThanEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to lt")); + } + case LESS_THAN_OR_EQUAL: { + verifyArgCount(argEvaluators, 1, "le"); + return new LessThanOrEqualEvaluator(toNumberEvaluator(subjectEvaluator), + toNumberEvaluator(argEvaluators.get(0), "first argument to le")); + } + case LENGTH: { + verifyArgCount(argEvaluators, 0, "length"); + return new LengthEvaluator(toStringEvaluator(subjectEvaluator)); + } + case TO_DATE: { + if (argEvaluators.isEmpty()) { + return new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator)); + } else if (subjectEvaluator.getResultType() == ResultType.STRING) { + return new StringToDateEvaluator(toStringEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0))); + } else { + return new NumberToDateEvaluator(toNumberEvaluator(subjectEvaluator)); + } + } + case TO_NUMBER: { + verifyArgCount(argEvaluators, 0, "toNumber"); + switch (subjectEvaluator.getResultType()) { + case STRING: + return new ToNumberEvaluator((StringEvaluator) subjectEvaluator); + case DATE: + return new DateToNumberEvaluator((DateEvaluator) subjectEvaluator); + default: + throw new AttributeExpressionLanguageParsingException(subjectEvaluator + " returns type " + subjectEvaluator.getResultType() + " but expected to get " + ResultType.STRING); + } + } + case TO_RADIX: { + if (argEvaluators.size() == 1) { + return new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0))); + } else { + return new ToRadixEvaluator((NumberEvaluator) subjectEvaluator, toNumberEvaluator(argEvaluators.get(0)), toNumberEvaluator(argEvaluators.get(1))); + } + } + case MOD: { + return new ModEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + } + case PLUS: { + return new PlusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + } + case MINUS: { + return new MinusEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + } + case MULTIPLY: { + return new MultiplyEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + } + case DIVIDE: { + return new DivideEvaluator(toNumberEvaluator(subjectEvaluator), toNumberEvaluator(argEvaluators.get(0))); + } + case INDEX_OF: { + verifyArgCount(argEvaluators, 1, "indexOf"); + return new IndexOfEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to indexOf")); + } + case LAST_INDEX_OF: { + verifyArgCount(argEvaluators, 1, "lastIndexOf"); + return new LastIndexOfEvaluator(toStringEvaluator(subjectEvaluator), + toStringEvaluator(argEvaluators.get(0), "first argument to lastIndexOf")); + } + case FORMAT: { + return new FormatEvaluator(toDateEvaluator(subjectEvaluator), toStringEvaluator(argEvaluators.get(0), "first argument of format")); + } + case OR: { + return new OrEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))); + } + case AND: { + return new AndEvaluator(toBooleanEvaluator(subjectEvaluator), toBooleanEvaluator(argEvaluators.get(0))); + } + case NOT: { + return new NotEvaluator(toBooleanEvaluator(subjectEvaluator)); + } + default: + throw new AttributeExpressionLanguageParsingException("Expected a Function-type expression but got " + tree.toString()); + } + } + + public static class Range { + + private final int start; + private final int end; + + public Range(final int start, final int end) { + this.start = start; + this.end = end; + } + + public int getStart() { + return start; + } + + public int getEnd() { + return end; + } + + @Override + public String toString() { + return start + " - " + end; + } + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java new file mode 100644 index 0000000000..49ef6efb51 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardAttributeExpression.java @@ -0,0 +1,65 @@ +/* + * 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.attribute.expression.language; + +import org.apache.nifi.expression.AttributeExpression; +import org.apache.nifi.expression.AttributeValueDecorator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.exception.ProcessException; + +public class StandardAttributeExpression implements AttributeExpression { + + private final Query query; + + public StandardAttributeExpression(final Query query) { + this.query = query; + } + + @Override + public ResultType getResultType() { + return query.getResultType(); + } + + @Override + public String evaluate() throws ProcessException { + return evaluate((AttributeValueDecorator) null); + } + + @Override + public String evaluate(final AttributeValueDecorator decorator) throws ProcessException { + return evaluate(null, decorator); + } + + @Override + public String evaluate(final FlowFile flowFile) throws ProcessException { + return evaluate(flowFile, null); + } + + @Override + public String evaluate(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException { + final Object evaluationResult = query.evaluate(flowFile).getValue(); + if (evaluationResult == null) { + return ""; + } + + String result = evaluationResult.toString(); + if (decorator != null) { + result = decorator.decorate(result); + } + return Query.unescape(result); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java new file mode 100644 index 0000000000..cec73d1748 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardExpressionLanguageCompiler.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language; + +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +import org.apache.nifi.expression.AttributeExpression; +import org.apache.nifi.expression.ExpressionLanguageCompiler; +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class StandardExpressionLanguageCompiler implements ExpressionLanguageCompiler { + + @Override + public AttributeExpression compile(final String expression) throws IllegalArgumentException { + try { + return new StandardAttributeExpression(Query.compile(expression)); + } catch (final AttributeExpressionLanguageParsingException e) { + throw new IllegalArgumentException(e.getMessage()); + } + } + + @Override + public boolean isValidExpression(final String expression) { + return Query.isValidExpression(expression); + } + + @Override + public String validateExpression(final String expression, final boolean allowSurroundingCharacters) { + try { + Query.validateExpression(expression, allowSurroundingCharacters); + return null; + } catch (final AttributeExpressionLanguageParsingException aelpe) { + return aelpe.getMessage(); + } + } + + @Override + public ResultType getResultType(final String expression) throws IllegalArgumentException { + try { + return Query.getResultType(expression); + } catch (final AttributeExpressionLanguageParsingException e) { + throw new IllegalArgumentException(e); + } + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java new file mode 100644 index 0000000000..0affb7f5c5 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java @@ -0,0 +1,83 @@ +/* + * 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.attribute.expression.language; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.expression.AttributeValueDecorator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.exception.ProcessException; + +import org.antlr.runtime.tree.Tree; + +public class StandardPreparedQuery implements PreparedQuery { + + private final List queryStrings; + private final Map trees; + + public StandardPreparedQuery(final List queryStrings, final Map trees) { + this.queryStrings = new ArrayList<>(queryStrings); + this.trees = new HashMap<>(trees); + } + + @Override + public String evaluateExpressions(Map attributes) throws ProcessException { + return evaluateExpressions(attributes, null); + } + + @Override + public String evaluateExpressions(final Map attributes, final AttributeValueDecorator decorator) throws ProcessException { + final StringBuilder sb = new StringBuilder(); + for (final String val : queryStrings) { + final Tree tree = trees.get(val); + if (tree == null) { + sb.append(val); + } else { + final String evaluated = Query.evaluateExpression(tree, val, attributes, decorator); + if (evaluated != null) { + sb.append(evaluated); + } + } + } + return sb.toString(); + } + + @Override + public String evaluateExpressions(final FlowFile flowFile, final AttributeValueDecorator decorator) throws ProcessException { + final Map expressionMap = Query.createExpressionMap(flowFile); + return evaluateExpressions(expressionMap, decorator); + } + + @Override + public String evaluateExpressions() throws ProcessException { + return evaluateExpressions((FlowFile) null, null); + } + + @Override + public String evaluateExpressions(final AttributeValueDecorator decorator) throws ProcessException { + return evaluateExpressions((FlowFile) null, decorator); + } + + @Override + public String evaluateExpressions(final FlowFile flowFile) throws ProcessException { + return evaluateExpressions(flowFile, null); + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java new file mode 100644 index 0000000000..376ddfe3d7 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanEvaluator.java @@ -0,0 +1,32 @@ +/* + * 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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public abstract class BooleanEvaluator implements Evaluator { + + @Override + public ResultType getResultType() { + return ResultType.BOOLEAN; + } + + @Override + public int getEvaluationsRemaining() { + return 0; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java new file mode 100644 index 0000000000..e5ef1139ed --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/BooleanQueryResult.java @@ -0,0 +1,43 @@ +/* + * 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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class BooleanQueryResult implements QueryResult { + + private final Boolean value; + + public BooleanQueryResult(final Boolean value) { + this.value = value; + } + + @Override + public Boolean getValue() { + return value; + } + + @Override + public ResultType getResultType() { + return ResultType.BOOLEAN; + } + + @Override + public String toString() { + return String.valueOf(getValue()); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java new file mode 100644 index 0000000000..7474b60077 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateEvaluator.java @@ -0,0 +1,34 @@ +/* + * 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.attribute.expression.language.evaluation; + +import java.util.Date; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public abstract class DateEvaluator implements Evaluator { + + @Override + public ResultType getResultType() { + return ResultType.DATE; + } + + @Override + public int getEvaluationsRemaining() { + return 0; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java new file mode 100644 index 0000000000..a77bbe9592 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/DateQueryResult.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation; + +import java.util.Date; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class DateQueryResult implements QueryResult { + + private final Date date; + + public DateQueryResult(final Date date) { + this.date = date; + } + + @Override + public Date getValue() { + return date; + } + + @Override + public ResultType getResultType() { + return ResultType.DATE; + } + + @Override + public String toString() { + return String.valueOf(getValue()); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java new file mode 100644 index 0000000000..6d164df25e --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/Evaluator.java @@ -0,0 +1,32 @@ +/* + * 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.attribute.expression.language.evaluation; + +import java.util.Map; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public interface Evaluator { + + QueryResult evaluate(Map attributes); + + ResultType getResultType(); + + int getEvaluationsRemaining(); + + Evaluator getSubjectEvaluator(); +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java new file mode 100644 index 0000000000..403bae358e --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberEvaluator.java @@ -0,0 +1,33 @@ +/* + * 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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public abstract class NumberEvaluator implements Evaluator { + + @Override + public ResultType getResultType() { + return ResultType.NUMBER; + } + + @Override + public int getEvaluationsRemaining() { + return 0; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java new file mode 100644 index 0000000000..fc3c961bc2 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/NumberQueryResult.java @@ -0,0 +1,43 @@ +/* + * 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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class NumberQueryResult implements QueryResult { + + private final Long value; + + public NumberQueryResult(final Long value) { + this.value = value; + } + + @Override + public Long getValue() { + return value; + } + + @Override + public ResultType getResultType() { + return ResultType.NUMBER; + } + + @Override + public String toString() { + return String.valueOf(getValue()); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.java new file mode 100644 index 0000000000..56bd76a25f --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/QueryResult.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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public interface QueryResult { + + T getValue(); + + ResultType getResultType(); +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java new file mode 100644 index 0000000000..1f4ff21393 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringEvaluator.java @@ -0,0 +1,32 @@ +/* + * 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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public abstract class StringEvaluator implements Evaluator { + + @Override + public ResultType getResultType() { + return ResultType.STRING; + } + + @Override + public int getEvaluationsRemaining() { + return 0; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java new file mode 100644 index 0000000000..c010bb168c --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/StringQueryResult.java @@ -0,0 +1,43 @@ +/* + * 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.attribute.expression.language.evaluation; + +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class StringQueryResult implements QueryResult { + + private final String value; + + public StringQueryResult(final String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + @Override + public ResultType getResultType() { + return ResultType.STRING; + } + + @Override + public String toString() { + return String.valueOf(getValue()); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java new file mode 100644 index 0000000000..a88b72b40e --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/BooleanCastEvaluator.java @@ -0,0 +1,50 @@ +/* + * 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.attribute.expression.language.evaluation.cast; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class BooleanCastEvaluator extends BooleanEvaluator { + + private final StringEvaluator subjectEvaluator; + + public BooleanCastEvaluator(final StringEvaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult result = subjectEvaluator.evaluate(attributes); + if (result.getValue() == null) { + return new BooleanQueryResult(null); + } + + return new BooleanQueryResult(Boolean.valueOf(result.getValue().trim())); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java new file mode 100644 index 0000000000..b6d758cbec --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/DateCastEvaluator.java @@ -0,0 +1,115 @@ +/* + * 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.attribute.expression.language.evaluation.cast; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageException; +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class DateCastEvaluator extends DateEvaluator { + + public static final String DATE_TO_STRING_FORMAT = "EEE MMM dd HH:mm:ss zzz yyyy"; + public static final Pattern DATE_TO_STRING_PATTERN = Pattern.compile("(?:[a-zA-Z]{3} ){2}\\d{2} \\d{2}\\:\\d{2}\\:\\d{2} (?:.*?) \\d{4}"); + + public static final String ALTERNATE_FORMAT_WITHOUT_MILLIS = "yyyy/MM/dd HH:mm:ss"; + public static final String ALTERNATE_FORMAT_WITH_MILLIS = "yyyy/MM/dd HH:mm:ss.SSS"; + public static final Pattern ALTERNATE_PATTERN = Pattern.compile("\\d{4}/\\d{2}/\\d{2} \\d{2}\\:\\d{2}\\:\\d{2}(\\.\\d{3})?"); + + public static final Pattern NUMBER_PATTERN = Pattern.compile("\\d+"); + + private final Evaluator subjectEvaluator; + + public DateCastEvaluator(final Evaluator subjectEvaluator) { + if (subjectEvaluator.getResultType() == ResultType.BOOLEAN) { + throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + subjectEvaluator.getResultType() + " to " + ResultType.DATE); + } + + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult result = subjectEvaluator.evaluate(attributes); + if (result.getValue() == null) { + return new DateQueryResult(null); + } + + switch (result.getResultType()) { + case DATE: + return (DateQueryResult) result; + case STRING: + final String value = ((StringQueryResult) result).getValue().trim(); + if (DATE_TO_STRING_PATTERN.matcher(value).matches()) { + final SimpleDateFormat sdf = new SimpleDateFormat(DATE_TO_STRING_FORMAT); + + try { + final Date date = sdf.parse(value); + return new DateQueryResult(date); + } catch (final ParseException pe) { + throw new AttributeExpressionLanguageException("Could not parse input as date", pe); + } + } else if (NUMBER_PATTERN.matcher(value).matches()) { + return new DateQueryResult(new Date(Long.valueOf(value))); + } else { + final Matcher altMatcher = ALTERNATE_PATTERN.matcher(value); + if (altMatcher.matches()) { + final String millisValue = altMatcher.group(1); + + final String format; + if (millisValue == null) { + format = ALTERNATE_FORMAT_WITHOUT_MILLIS; + } else { + format = ALTERNATE_FORMAT_WITH_MILLIS; + } + + final SimpleDateFormat sdf = new SimpleDateFormat(format); + + try { + final Date date = sdf.parse(value); + return new DateQueryResult(date); + } catch (final ParseException pe) { + throw new AttributeExpressionLanguageException("Could not parse input as date", pe); + } + } else { + throw new AttributeExpressionLanguageException("Could not implicitly convert input to DATE: " + value); + } + } + case NUMBER: + return new DateQueryResult(new Date((Long) result.getValue())); + default: + return new DateQueryResult(null); + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java new file mode 100644 index 0000000000..0447d483a8 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/NumberCastEvaluator.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.cast; + +import java.util.Map; +import java.util.regex.Pattern; + +import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class NumberCastEvaluator extends NumberEvaluator { + + private final Evaluator subjectEvaluator; + private static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+"); + + public NumberCastEvaluator(final Evaluator subjectEvaluator) { + if (subjectEvaluator.getResultType() == ResultType.BOOLEAN) { + throw new AttributeExpressionLanguageParsingException("Cannot implicitly convert Data Type " + subjectEvaluator.getResultType() + " to " + ResultType.NUMBER); + } + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult result = subjectEvaluator.evaluate(attributes); + if (result.getValue() == null) { + return new NumberQueryResult(null); + } + + switch (result.getResultType()) { + case NUMBER: + return (NumberQueryResult) result; + case STRING: + final String trimmed = ((StringQueryResult) result).getValue().trim(); + if (NUMBER_PATTERN.matcher(trimmed).matches()) { + return new NumberQueryResult(Long.valueOf(trimmed)); + } else { + return new NumberQueryResult(null); + } + case DATE: + return new NumberQueryResult(((DateQueryResult) result).getValue().getTime()); + default: + return new NumberQueryResult(null); + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java new file mode 100644 index 0000000000..1a3732a25d --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/cast/StringCastEvaluator.java @@ -0,0 +1,49 @@ +/* + * 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.attribute.expression.language.evaluation.cast; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class StringCastEvaluator extends StringEvaluator { + + private final Evaluator subjectEvaluator; + + public StringCastEvaluator(final Evaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult result = subjectEvaluator.evaluate(attributes); + if (result.getValue() == null) { + return new StringQueryResult(null); + } + + return new StringQueryResult(String.valueOf(result.getValue())); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java new file mode 100644 index 0000000000..9ed8f06516 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AndEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class AndEvaluator extends BooleanEvaluator { + + private final BooleanEvaluator subjectEvaluator; + private final BooleanEvaluator rhsEvaluator; + + public AndEvaluator(final BooleanEvaluator subjectEvaluator, final BooleanEvaluator rhsEvaluator) { + this.subjectEvaluator = subjectEvaluator; + this.rhsEvaluator = rhsEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult subjectValue = subjectEvaluator.evaluate(attributes); + if (subjectValue == null) { + return new BooleanQueryResult(null); + } + + if (Boolean.FALSE.equals(subjectValue.getValue())) { + return new BooleanQueryResult(false); + } + + final QueryResult rhsValue = rhsEvaluator.evaluate(attributes); + if (rhsValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(rhsValue.getValue()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java new file mode 100644 index 0000000000..bd5e81a087 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AppendEvaluator.java @@ -0,0 +1,50 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class AppendEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator appendEvaluator; + + public AppendEvaluator(final StringEvaluator subject, final StringEvaluator append) { + this.subject = subject; + this.appendEvaluator = append; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + final String appendValue = appendEvaluator.evaluate(attributes).getValue(); + + final String result = (subjectValue == null ? "" : subjectValue) + + (appendValue == null ? "" : appendValue); + return new StringQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java new file mode 100644 index 0000000000..1b4b74ecce --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/AttributeEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class AttributeEvaluator extends StringEvaluator { + + private final StringEvaluator nameEvaluator; + + public AttributeEvaluator(final StringEvaluator nameEvaluator) { + this.nameEvaluator = nameEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String nameValue = nameEvaluator.evaluate(attributes).getValue(); + final String attributeValue = attributes.get(nameValue); + return new StringQueryResult(attributeValue); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java new file mode 100644 index 0000000000..99802f6347 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ContainsEvaluator.java @@ -0,0 +1,53 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class ContainsEvaluator extends BooleanEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + + public ContainsEvaluator(final StringEvaluator subject, final StringEvaluator search) { + this.subject = subject; + this.search = search; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final String searchString = search.evaluate(attributes).getValue(); + return new BooleanQueryResult(searchString == null ? false : subjectValue.contains(searchString)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java new file mode 100644 index 0000000000..0594c4a882 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DateToNumberEvaluator.java @@ -0,0 +1,50 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Date; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class DateToNumberEvaluator extends NumberEvaluator { + + private final DateEvaluator subjectEvaluator; + + public DateToNumberEvaluator(final DateEvaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult subjectResult = subjectEvaluator.evaluate(attributes); + if (subjectResult.getValue() == null) { + return new NumberQueryResult(null); + } + + return new NumberQueryResult(subjectResult.getValue().getTime()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.java new file mode 100644 index 0000000000..c7583cbb75 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/DivideEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class DivideEvaluator extends NumberEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator divideValue; + + public DivideEvaluator(final NumberEvaluator subject, final NumberEvaluator divideValue) { + this.subject = subject; + this.divideValue = divideValue; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(null); + } + + final Long divide = divideValue.evaluate(attributes).getValue(); + if (divide == null) { + return new NumberQueryResult(null); + } + + final long result = subjectValue / divide; + return new NumberQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java new file mode 100644 index 0000000000..c31c2b1230 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EndsWithEvaluator.java @@ -0,0 +1,53 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class EndsWithEvaluator extends BooleanEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + + public EndsWithEvaluator(final StringEvaluator subject, final StringEvaluator search) { + this.subject = subject; + this.search = search; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final String searchString = search.evaluate(attributes).getValue(); + return new BooleanQueryResult(searchString == null ? false : subjectValue.endsWith(searchString)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java new file mode 100644 index 0000000000..ec67b0feb9 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsEvaluator.java @@ -0,0 +1,89 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Date; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.expression.AttributeExpression.ResultType; + +public class EqualsEvaluator extends BooleanEvaluator { + + private final Evaluator subject; + private final Evaluator compareTo; + + public EqualsEvaluator(final Evaluator subject, final Evaluator compareTo) { + this.subject = subject; + this.compareTo = compareTo; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Object a = subject.evaluate(attributes).getValue(); + if (a == null) { + return new BooleanQueryResult(false); + } + + final Object b = compareTo.evaluate(attributes).getValue(); + if (b == null) { + return new BooleanQueryResult(false); + } + + if (subject.getResultType() == compareTo.getResultType()) { + return new BooleanQueryResult(a.equals(b)); + } + + final String normalizedSubjectValue = normalizeValue(subject.getResultType(), a); + if (normalizedSubjectValue == null) { + return new BooleanQueryResult(false); + } + + final String normalizedCompareToValue = normalizeValue(compareTo.getResultType(), b); + if (normalizedCompareToValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(normalizedSubjectValue.equals(normalizedCompareToValue)); + } + + private String normalizeValue(final ResultType type, final Object value) { + if (value == null) { + return null; + } + + switch (type) { + case STRING: + return (String) value; + case DATE: + return String.valueOf(((Date) value).getTime()); + case BOOLEAN: + case NUMBER: + default: + return String.valueOf(value); + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java new file mode 100644 index 0000000000..344395b00a --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/EqualsIgnoreCaseEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class EqualsIgnoreCaseEvaluator extends BooleanEvaluator { + + private final Evaluator subject; + private final Evaluator compareTo; + + public EqualsIgnoreCaseEvaluator(final Evaluator subject, final Evaluator compareTo) { + this.subject = subject; + this.compareTo = compareTo; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Object a = subject.evaluate(attributes).getValue(); + if (a == null) { + return new BooleanQueryResult(false); + } + + final Object b = compareTo.evaluate(attributes).getValue(); + if (b == null) { + return new BooleanQueryResult(false); + } + + if (a instanceof String && b instanceof String) { + return new BooleanQueryResult(((String) a).equalsIgnoreCase((String) b)); + } + + return new BooleanQueryResult(a.equals(b)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java new file mode 100644 index 0000000000..b377c4ad3e --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FindEvaluator.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; +import java.util.regex.Pattern; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; + +public class FindEvaluator extends BooleanEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + + private final Pattern compiledPattern; + + public FindEvaluator(final StringEvaluator subject, final StringEvaluator search) { + this.subject = subject; + this.search = search; + + // if the search string is a literal, we don't need to evaluate it each time; we can just + // pre-compile it. Otherwise, it must be compiled every time. + if (search instanceof StringLiteralEvaluator) { + this.compiledPattern = Pattern.compile(search.evaluate(null).getValue()); + } else { + this.compiledPattern = null; + } + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + final Pattern pattern; + if (compiledPattern == null) { + pattern = Pattern.compile(search.evaluate(attributes).getValue()); + } else { + pattern = compiledPattern; + } + + final boolean found = pattern.matcher(subjectValue).find(); + + return new BooleanQueryResult(found); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java new file mode 100644 index 0000000000..ccb1a89d9e --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/FormatEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class FormatEvaluator extends StringEvaluator { + + private final DateEvaluator subject; + private final StringEvaluator format; + + public FormatEvaluator(final DateEvaluator subject, final StringEvaluator format) { + this.subject = subject; + this.format = format; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Date subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(null); + } + + final QueryResult formatResult = format.evaluate(attributes); + final String format = formatResult.getValue(); + if (format == null) { + return null; + } + + return new StringQueryResult(new SimpleDateFormat(format).format(subjectValue)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java new file mode 100644 index 0000000000..6c712bbda0 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class GreaterThanEvaluator extends BooleanEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator comparison; + + public GreaterThanEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + this.subject = subject; + this.comparison = comparison; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final Long comparisonValue = comparison.evaluate(attributes).getValue(); + if (comparisonValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(subjectValue > comparisonValue); + } + + ; + + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java new file mode 100644 index 0000000000..98951f27e1 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/GreaterThanOrEqualEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class GreaterThanOrEqualEvaluator extends BooleanEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator comparison; + + public GreaterThanOrEqualEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + this.subject = subject; + this.comparison = comparison; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final Long comparisonValue = comparison.evaluate(attributes).getValue(); + if (comparisonValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(subjectValue >= comparisonValue); + } + + ; + + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java new file mode 100644 index 0000000000..c0b1134596 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/HostnameEvaluator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class HostnameEvaluator extends StringEvaluator { + + private final StringQueryResult hostname; + + public HostnameEvaluator(final boolean preferFQDN) throws UnknownHostException { + String address; + + if (preferFQDN) { + try { + address = InetAddress.getLocalHost().getCanonicalHostName(); + } catch (final Exception e) { + address = InetAddress.getLocalHost().getHostName(); + } + } else { + address = InetAddress.getLocalHost().getHostName(); + } + + hostname = new StringQueryResult(address); + } + + @Override + public QueryResult evaluate(final Map attributes) { + return hostname; + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.java new file mode 100644 index 0000000000..21f9fe8080 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IPEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class IPEvaluator extends StringEvaluator { + + private final StringQueryResult ipAddress; + + public IPEvaluator() throws UnknownHostException { + ipAddress = new StringQueryResult(InetAddress.getLocalHost().getHostAddress()); + } + + @Override + public QueryResult evaluate(final Map attributes) { + return ipAddress; + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java new file mode 100644 index 0000000000..80422cb67b --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IndexOfEvaluator.java @@ -0,0 +1,53 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class IndexOfEvaluator extends NumberEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator indexEvaluator; + + public IndexOfEvaluator(final StringEvaluator subject, final StringEvaluator indexEvaluator) { + this.subject = subject; + this.indexEvaluator = indexEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(-1L); + } + final String indexEvalValue = indexEvaluator.evaluate(attributes).getValue(); + + return new NumberQueryResult((long) subjectValue.indexOf(indexEvalValue)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java new file mode 100644 index 0000000000..a9d678c790 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/IsNullEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class IsNullEvaluator extends BooleanEvaluator { + + private final Evaluator subject; + + public IsNullEvaluator(final Evaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Object subjectValue = subject.evaluate(attributes).getValue(); + return new BooleanQueryResult(subjectValue == null); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java new file mode 100644 index 0000000000..2b670935b1 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LastIndexOfEvaluator.java @@ -0,0 +1,53 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class LastIndexOfEvaluator extends NumberEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator indexEvaluator; + + public LastIndexOfEvaluator(final StringEvaluator subject, final StringEvaluator indexEvaluator) { + this.subject = subject; + this.indexEvaluator = indexEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(-1L); + } + final String indexEvalValue = indexEvaluator.evaluate(attributes).getValue(); + + return new NumberQueryResult((long) subjectValue.lastIndexOf(indexEvalValue)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.java new file mode 100644 index 0000000000..dec1f8f4b5 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LengthEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class LengthEvaluator extends NumberEvaluator { + + private final StringEvaluator subject; + + public LengthEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + return new NumberQueryResult((long) ((subjectValue == null) ? 0 : subjectValue.length())); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java new file mode 100644 index 0000000000..9a589103c6 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class LessThanEvaluator extends BooleanEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator comparison; + + public LessThanEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + this.subject = subject; + this.comparison = comparison; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final Long comparisonValue = comparison.evaluate(attributes).getValue(); + if (comparisonValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(subjectValue < comparisonValue); + } + + ; + + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java new file mode 100644 index 0000000000..10f3f6aad8 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/LessThanOrEqualEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class LessThanOrEqualEvaluator extends BooleanEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator comparison; + + public LessThanOrEqualEvaluator(final NumberEvaluator subject, final NumberEvaluator comparison) { + this.subject = subject; + this.comparison = comparison; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final Long comparisonValue = comparison.evaluate(attributes).getValue(); + if (comparisonValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(subjectValue <= comparisonValue); + } + + ; + + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java new file mode 100644 index 0000000000..1bb86a5c34 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MatchesEvaluator.java @@ -0,0 +1,71 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; +import java.util.regex.Pattern; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; + +public class MatchesEvaluator extends BooleanEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + + private final Pattern compiledPattern; + + public MatchesEvaluator(final StringEvaluator subject, final StringEvaluator search) { + this.subject = subject; + this.search = search; + + // if the search string is a literal, we don't need to evaluate it each time; we can just + // pre-compile it. Otherwise, it must be compiled every time. + if (search instanceof StringLiteralEvaluator) { + this.compiledPattern = Pattern.compile(search.evaluate(null).getValue()); + } else { + this.compiledPattern = null; + } + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + final Pattern pattern; + if (compiledPattern == null) { + pattern = Pattern.compile(search.evaluate(attributes).getValue()); + } else { + pattern = compiledPattern; + } + + final boolean matches = pattern.matcher(subjectValue).matches(); + return new BooleanQueryResult(matches); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.java new file mode 100644 index 0000000000..066597966d --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MinusEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class MinusEvaluator extends NumberEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator minusValue; + + public MinusEvaluator(final NumberEvaluator subject, final NumberEvaluator minusValue) { + this.subject = subject; + this.minusValue = minusValue; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(null); + } + + final Long minus = minusValue.evaluate(attributes).getValue(); + if (minus == null) { + return new NumberQueryResult(null); + } + + final long result = subjectValue - minus; + return new NumberQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.java new file mode 100644 index 0000000000..6b7b8388df --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ModEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class ModEvaluator extends NumberEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator modValue; + + public ModEvaluator(final NumberEvaluator subject, final NumberEvaluator modValue) { + this.subject = subject; + this.modValue = modValue; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(null); + } + + final Long mod = modValue.evaluate(attributes).getValue(); + if (mod == null) { + return new NumberQueryResult(null); + } + + final long result = subjectValue % mod; + return new NumberQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.java new file mode 100644 index 0000000000..a3b03ed94e --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/MultiplyEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class MultiplyEvaluator extends NumberEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator multiplyValue; + + public MultiplyEvaluator(final NumberEvaluator subject, final NumberEvaluator multiplyValue) { + this.subject = subject; + this.multiplyValue = multiplyValue; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(null); + } + + final Long multiply = multiplyValue.evaluate(attributes).getValue(); + if (multiply == null) { + return new NumberQueryResult(null); + } + + final long result = subjectValue * multiply; + return new NumberQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java new file mode 100644 index 0000000000..327e620890 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotEvaluator.java @@ -0,0 +1,49 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class NotEvaluator extends BooleanEvaluator { + + private final BooleanEvaluator subjectEvaluator; + + public NotEvaluator(final BooleanEvaluator subjectEvaluator) { + this.subjectEvaluator = subjectEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult subjectValue = subjectEvaluator.evaluate(attributes); + if (subjectValue == null) { + return new BooleanQueryResult(null); + } + + return new BooleanQueryResult(!subjectValue.getValue()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java new file mode 100644 index 0000000000..126fbfa6a5 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NotNullEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class NotNullEvaluator extends BooleanEvaluator { + + private final Evaluator subject; + + public NotNullEvaluator(final Evaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Object subjectValue = subject.evaluate(attributes).getValue(); + return new BooleanQueryResult(subjectValue != null); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java new file mode 100644 index 0000000000..bfde6b0cb1 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NowEvaluator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Date; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class NowEvaluator extends DateEvaluator { + + @Override + public QueryResult evaluate(final Map attributes) { + return new DateQueryResult(new Date()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java new file mode 100644 index 0000000000..672d855dc9 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/NumberToDateEvaluator.java @@ -0,0 +1,52 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Date; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class NumberToDateEvaluator extends DateEvaluator { + + private final NumberEvaluator subject; + + public NumberToDateEvaluator(final NumberEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult result = subject.evaluate(attributes); + final Long value = result.getValue(); + if (value == null) { + return null; + } + + return new DateQueryResult(new Date(value)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java new file mode 100644 index 0000000000..11d3fa9d7c --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OneUpSequenceEvaluator.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class OneUpSequenceEvaluator extends NumberEvaluator { + + private static final AtomicLong value = new AtomicLong(0L); + + @Override + public QueryResult evaluate(final Map attributes) { + return new NumberQueryResult(value.getAndIncrement()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java new file mode 100644 index 0000000000..ab3a16ce1c --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/OrEvaluator.java @@ -0,0 +1,60 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class OrEvaluator extends BooleanEvaluator { + + private final BooleanEvaluator subjectEvaluator; + private final BooleanEvaluator rhsEvaluator; + + public OrEvaluator(final BooleanEvaluator subjectEvaluator, final BooleanEvaluator rhsEvaluator) { + this.subjectEvaluator = subjectEvaluator; + this.rhsEvaluator = rhsEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final QueryResult subjectValue = subjectEvaluator.evaluate(attributes); + if (subjectValue == null) { + return new BooleanQueryResult(null); + } + + if (Boolean.TRUE.equals(subjectValue.getValue())) { + return new BooleanQueryResult(true); + } + + final QueryResult rhsValue = rhsEvaluator.evaluate(attributes); + if (rhsValue == null) { + return new BooleanQueryResult(false); + } + + return new BooleanQueryResult(rhsValue.getValue()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subjectEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.java new file mode 100644 index 0000000000..dff693dd0a --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PlusEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class PlusEvaluator extends NumberEvaluator { + + private final NumberEvaluator subject; + private final NumberEvaluator plusValue; + + public PlusEvaluator(final NumberEvaluator subject, final NumberEvaluator plusValue) { + this.subject = subject; + this.plusValue = plusValue; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new NumberQueryResult(null); + } + + final Long plus = plusValue.evaluate(attributes).getValue(); + if (plus == null) { + return new NumberQueryResult(null); + } + + final long result = subjectValue + plus; + return new NumberQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java new file mode 100644 index 0000000000..5b66b8fcaa --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/PrependEvaluator.java @@ -0,0 +1,50 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class PrependEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator prependEvaluator; + + public PrependEvaluator(final StringEvaluator subject, final StringEvaluator prepend) { + this.subject = subject; + this.prependEvaluator = prepend; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + final String prependValue = prependEvaluator.evaluate(attributes).getValue(); + + final String result = (prependValue == null ? "" : prependValue) + (subjectValue == null ? "" : subjectValue); + return new StringQueryResult(result); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java new file mode 100644 index 0000000000..19ed63ef71 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceAllEvaluator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ReplaceAllEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + private final StringEvaluator replacement; + + public ReplaceAllEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) { + this.subject = subject; + this.search = search; + this.replacement = replacement; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(null); + } + final String searchValue = search.evaluate(attributes).getValue(); + final String replacementValue = replacement.evaluate(attributes).getValue(); + + return new StringQueryResult(subjectValue.replaceAll(searchValue, replacementValue)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java new file mode 100644 index 0000000000..d6a7c5b7cc --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceEvaluator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ReplaceEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + private final StringEvaluator replacement; + + public ReplaceEvaluator(final StringEvaluator subject, final StringEvaluator search, final StringEvaluator replacement) { + this.subject = subject; + this.search = search; + this.replacement = replacement; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(null); + } + final String searchValue = search.evaluate(attributes).getValue(); + final String replacementValue = replacement.evaluate(attributes).getValue(); + + return new StringQueryResult(subjectValue.replace(searchValue, replacementValue)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java new file mode 100644 index 0000000000..91333b727b --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ReplaceNullEvaluator.java @@ -0,0 +1,47 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ReplaceNullEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator resultEvaluator; + + public ReplaceNullEvaluator(final StringEvaluator subject, final StringEvaluator resultEvaluator) { + this.subject = subject; + this.resultEvaluator = resultEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + return new StringQueryResult(subjectValue == null ? resultEvaluator.evaluate(attributes).getValue() : subjectValue); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java new file mode 100644 index 0000000000..c6a7eb7c72 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StartsWithEvaluator.java @@ -0,0 +1,53 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class StartsWithEvaluator extends BooleanEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator search; + + public StartsWithEvaluator(final StringEvaluator subject, final StringEvaluator search) { + this.subject = subject; + this.search = search; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new BooleanQueryResult(false); + } + + final String searchString = search.evaluate(attributes).getValue(); + return new BooleanQueryResult(searchString == null ? false : subjectValue.startsWith(searchString)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java new file mode 100644 index 0000000000..a4e21a44a7 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/StringToDateEvaluator.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.DateEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.DateQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.exception.IllegalAttributeException; + +public class StringToDateEvaluator extends DateEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator format; + + public StringToDateEvaluator(final StringEvaluator subject, final StringEvaluator format) { + this.subject = subject; + this.format = format; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + final String formatValue = format.evaluate(attributes).getValue(); + if (subjectValue == null || formatValue == null) { + return new DateQueryResult(null); + } + + try { + return new DateQueryResult(new SimpleDateFormat(formatValue).parse(subjectValue)); + } catch (final ParseException e) { + throw new IllegalAttributeException("Cannot parse attribute value as a date; date format: " + + formatValue + "; attribute value: " + subjectValue); + } catch (final IllegalArgumentException e) { + throw new IllegalAttributeException("Invalid date format: " + formatValue); + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java new file mode 100644 index 0000000000..a8b7357d94 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterEvaluator.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class SubstringAfterEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator afterEvaluator; + + public SubstringAfterEvaluator(final StringEvaluator subject, final StringEvaluator afterEvaluator) { + this.subject = subject; + this.afterEvaluator = afterEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(""); + } + final String afterValue = afterEvaluator.evaluate(attributes).getValue(); + if (afterValue == null || afterValue.length() == 0) { + return new StringQueryResult(subjectValue); + } + + final int index = subjectValue.indexOf(afterValue); + if (index < 0) { + return new StringQueryResult(subjectValue); + } + return new StringQueryResult(subjectValue.substring(index + afterValue.length())); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java new file mode 100644 index 0000000000..3dac40ebe9 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringAfterLastEvaluator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class SubstringAfterLastEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator afterEvaluator; + + public SubstringAfterLastEvaluator(final StringEvaluator subject, final StringEvaluator afterEvaluator) { + this.subject = subject; + this.afterEvaluator = afterEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(""); + } + final String afterValue = afterEvaluator.evaluate(attributes).getValue(); + final int index = subjectValue.lastIndexOf(afterValue); + if (index < 0 || index >= subjectValue.length()) { + return new StringQueryResult(subjectValue); + } + return new StringQueryResult(subjectValue.substring(index + afterValue.length())); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java new file mode 100644 index 0000000000..92b136cd83 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeEvaluator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class SubstringBeforeEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator beforeEvaluator; + + public SubstringBeforeEvaluator(final StringEvaluator subject, final StringEvaluator beforeEvaluator) { + this.subject = subject; + this.beforeEvaluator = beforeEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(""); + } + final String beforeValue = beforeEvaluator.evaluate(attributes).getValue(); + if (beforeValue == null || beforeValue.length() == 0) { + return new StringQueryResult(subjectValue); + } + final int index = subjectValue.indexOf(beforeValue); + if (index < 0) { + return new StringQueryResult(subjectValue); + } + return new StringQueryResult(subjectValue.substring(0, index)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java new file mode 100644 index 0000000000..818a3dc9f0 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringBeforeLastEvaluator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class SubstringBeforeLastEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final StringEvaluator beforeEvaluator; + + public SubstringBeforeLastEvaluator(final StringEvaluator subject, final StringEvaluator beforeEvaluator) { + this.subject = subject; + this.beforeEvaluator = beforeEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(""); + } + final String beforeValue = beforeEvaluator.evaluate(attributes).getValue(); + final int index = subjectValue.lastIndexOf(beforeValue); + if (index < 0) { + return new StringQueryResult(subjectValue); + } + return new StringQueryResult(subjectValue.substring(0, index)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java new file mode 100644 index 0000000000..da11c45464 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/SubstringEvaluator.java @@ -0,0 +1,65 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class SubstringEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + private final NumberEvaluator startIndex; + private final NumberEvaluator endIndex; + + public SubstringEvaluator(final StringEvaluator subject, final NumberEvaluator startIndex, final NumberEvaluator endIndex) { + this.subject = subject; + this.startIndex = startIndex; + this.endIndex = endIndex; + } + + public SubstringEvaluator(final StringEvaluator subject, final NumberEvaluator startIndex) { + this.subject = subject; + this.startIndex = startIndex; + this.endIndex = null; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(""); + } + final int startIndexValue = startIndex.evaluate(attributes).getValue().intValue(); + if (endIndex == null) { + return new StringQueryResult(subjectValue.substring(startIndexValue)); + } else { + final int endIndexValue = endIndex.evaluate(attributes).getValue().intValue(); + return new StringQueryResult(subjectValue.substring(startIndexValue, endIndexValue)); + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java new file mode 100644 index 0000000000..7de01eb2f3 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToLowerEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ToLowerEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + + public ToLowerEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + return new StringQueryResult((subjectValue == null) ? null : subjectValue.toLowerCase()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.java new file mode 100644 index 0000000000..1d4664acfe --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToNumberEvaluator.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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public class ToNumberEvaluator extends NumberEvaluator { + + private final StringEvaluator subject; + + public ToNumberEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + return new NumberQueryResult((subjectValue == null || subjectValue.trim().isEmpty()) ? null : Long.valueOf(subjectValue)); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java new file mode 100644 index 0000000000..aab094dcc5 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToRadixEvaluator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Arrays; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ToRadixEvaluator extends StringEvaluator { + + private final NumberEvaluator numberEvaluator; + private final NumberEvaluator radixEvaluator; + private final NumberEvaluator minimumWidthEvaluator; + + public ToRadixEvaluator(final NumberEvaluator subject, final NumberEvaluator radixEvaluator) { + this(subject, radixEvaluator, null); + } + + public ToRadixEvaluator(final NumberEvaluator subject, final NumberEvaluator radixEvaluator, final NumberEvaluator minimumWidthEvaluator) { + this.numberEvaluator = subject; + this.radixEvaluator = radixEvaluator; + this.minimumWidthEvaluator = minimumWidthEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Long result = numberEvaluator.evaluate(attributes).getValue(); + if (result == null) { + return new StringQueryResult(null); + } + + final Long radix = radixEvaluator.evaluate(attributes).getValue(); + if (radix == null) { + return new StringQueryResult(null); + } + + String stringValue = Long.toString(result.longValue(), radix.intValue()); + if (minimumWidthEvaluator != null) { + final Long minimumWidth = minimumWidthEvaluator.evaluate(attributes).getValue(); + if (minimumWidth != null) { + final int paddingWidth = minimumWidth.intValue() - stringValue.length(); + if (paddingWidth > 0) { + final char[] padChars = new char[paddingWidth]; + Arrays.fill(padChars, '0'); + stringValue = String.valueOf(padChars) + stringValue; + } + } + } + + return new StringQueryResult(stringValue); + } + + @Override + public Evaluator getSubjectEvaluator() { + return numberEvaluator; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java new file mode 100644 index 0000000000..2f7fbe360f --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToStringEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ToStringEvaluator extends StringEvaluator { + + private final Evaluator subject; + + public ToStringEvaluator(final Evaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final Object result = subject.evaluate(attributes).getValue(); + return new StringQueryResult(result == null ? null : result.toString()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java new file mode 100644 index 0000000000..29bc61d9a4 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/ToUpperEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class ToUpperEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + + public ToUpperEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + return new StringQueryResult((subjectValue == null) ? null : subjectValue.toUpperCase()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java new file mode 100644 index 0000000000..b9926d7e34 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/TrimEvaluator.java @@ -0,0 +1,45 @@ +/* + * 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.attribute.expression.language.evaluation.functions; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class TrimEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + + public TrimEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + return new StringQueryResult(subjectValue == null ? null : subjectValue.trim()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java new file mode 100644 index 0000000000..6c3ba14bc6 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlDecodeEvaluator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class UrlDecodeEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + + public UrlDecodeEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(null); + } + + try { + return new StringQueryResult(URLDecoder.decode(subjectValue, "UTF-8")); + } catch (UnsupportedEncodingException e) { + return null; // won't happen. It's UTF-8 + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java new file mode 100644 index 0000000000..d1e2caa5b6 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UrlEncodeEvaluator.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.io.UnsupportedEncodingException; +import java.net.URLEncoder; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class UrlEncodeEvaluator extends StringEvaluator { + + private final StringEvaluator subject; + + public UrlEncodeEvaluator(final StringEvaluator subject) { + this.subject = subject; + } + + @Override + public QueryResult evaluate(final Map attributes) { + final String subjectValue = subject.evaluate(attributes).getValue(); + if (subjectValue == null) { + return new StringQueryResult(null); + } + + try { + return new StringQueryResult(URLEncoder.encode(subjectValue, "UTF-8")); + } catch (UnsupportedEncodingException e) { + return null; // won't happen. It's UTF-8 + } + } + + @Override + public Evaluator getSubjectEvaluator() { + return subject; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java new file mode 100644 index 0000000000..faa8f0ade1 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/functions/UuidEvaluator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.functions; + +import java.util.Map; +import java.util.UUID; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class UuidEvaluator extends StringEvaluator { + + @Override + public QueryResult evaluate(final Map attributes) { + return new StringQueryResult(UUID.randomUUID().toString()); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java new file mode 100644 index 0000000000..a085423396 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/BooleanLiteralEvaluator.java @@ -0,0 +1,44 @@ +/* + * 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.attribute.expression.language.evaluation.literals; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class BooleanLiteralEvaluator extends BooleanEvaluator { + + private final boolean value; + + public BooleanLiteralEvaluator(final boolean value) { + this.value = value; + } + + @Override + public QueryResult evaluate(final Map attributes) { + return new BooleanQueryResult(value); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java new file mode 100644 index 0000000000..d7569e026f --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/NumberLiteralEvaluator.java @@ -0,0 +1,44 @@ +/* + * 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.attribute.expression.language.evaluation.literals; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.NumberQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class NumberLiteralEvaluator extends NumberEvaluator { + + private final long literal; + + public NumberLiteralEvaluator(final String value) { + this.literal = Long.parseLong(value); + } + + @Override + public QueryResult evaluate(final Map attributes) { + return new NumberQueryResult(literal); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java new file mode 100644 index 0000000000..d739ac7c68 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/literals/StringLiteralEvaluator.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.literals; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class StringLiteralEvaluator extends StringEvaluator { + + private final String value; + + public StringLiteralEvaluator(final String value) { + // need to escape characters after backslashes + final StringBuilder sb = new StringBuilder(); + boolean lastCharIsBackslash = false; + for (int i = 0; i < value.length(); i++) { + final char c = value.charAt(i); + + if (lastCharIsBackslash) { + switch (c) { + case 'n': + sb.append("\n"); + break; + case 'r': + sb.append("\r"); + break; + case '\\': + sb.append("\\"); + break; + case 't': + sb.append("\\t"); + break; + default: + sb.append("\\").append(c); + break; + } + + lastCharIsBackslash = false; + } else if (c == '\\') { + lastCharIsBackslash = true; + } else { + sb.append(c); + } + } + + this.value = sb.toString(); + } + + @Override + public QueryResult evaluate(final Map attributes) { + return new StringQueryResult(value); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java new file mode 100644 index 0000000000..d9dd4d3a1b --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AllAttributesEvaluator.java @@ -0,0 +1,68 @@ +/* + * 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.attribute.expression.language.evaluation.selection; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class AllAttributesEvaluator extends BooleanEvaluator { + + private final BooleanEvaluator booleanEvaluator; + private final MultiAttributeEvaluator multiAttributeEvaluator; + + public AllAttributesEvaluator(final BooleanEvaluator booleanEvaluator, final MultiAttributeEvaluator multiAttributeEvaluator) { + this.booleanEvaluator = booleanEvaluator; + this.multiAttributeEvaluator = multiAttributeEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + QueryResult attributeValueQuery = booleanEvaluator.evaluate(attributes); + Boolean result = attributeValueQuery.getValue(); + if (result == null) { + return new BooleanQueryResult(false); + } + + if (!result) { + return new BooleanQueryResult(false); + } + + while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) { + attributeValueQuery = booleanEvaluator.evaluate(attributes); + result = attributeValueQuery.getValue(); + if (result != null && !result) { + return attributeValueQuery; + } + } + + return new BooleanQueryResult(true); + } + + @Override + public int getEvaluationsRemaining() { + return 0; + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java new file mode 100644 index 0000000000..9192958164 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyAttributeEvaluator.java @@ -0,0 +1,68 @@ +/* + * 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.attribute.expression.language.evaluation.selection; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.BooleanEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.BooleanQueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; + +public class AnyAttributeEvaluator extends BooleanEvaluator { + + private final BooleanEvaluator booleanEvaluator; + private final MultiAttributeEvaluator multiAttributeEvaluator; + + public AnyAttributeEvaluator(final BooleanEvaluator booleanEvaluator, final MultiAttributeEvaluator multiAttributeEvaluator) { + this.booleanEvaluator = booleanEvaluator; + this.multiAttributeEvaluator = multiAttributeEvaluator; + } + + @Override + public QueryResult evaluate(final Map attributes) { + QueryResult attributeValueQuery = booleanEvaluator.evaluate(attributes); + Boolean result = attributeValueQuery.getValue(); + if (result == null) { + return new BooleanQueryResult(false); + } + + if (result) { + return new BooleanQueryResult(true); + } + + while (multiAttributeEvaluator.getEvaluationsRemaining() > 0) { + attributeValueQuery = booleanEvaluator.evaluate(attributes); + result = attributeValueQuery.getValue(); + if (result != null && result) { + return attributeValueQuery; + } + } + + return new BooleanQueryResult(false); + } + + @Override + public int getEvaluationsRemaining() { + return 0; + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java new file mode 100644 index 0000000000..8c07278c3a --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/AnyMatchingAttributeEvaluator.java @@ -0,0 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.selection; + +public class AnyMatchingAttributeEvaluator { + +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java new file mode 100644 index 0000000000..209c86f3a5 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/DelineatedAttributeEvaluator.java @@ -0,0 +1,83 @@ +/* + * 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.attribute.expression.language.evaluation.selection; + +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class DelineatedAttributeEvaluator extends MultiAttributeEvaluator { + + private final StringEvaluator subjectEvaluator; + private final StringEvaluator delimiterEvaluator; + private final int evaluationType; + private String[] delineatedValues; + private int evaluationCount = 0; + private int evaluationsLeft = 1; + + public DelineatedAttributeEvaluator(final StringEvaluator subjectEvaluator, final StringEvaluator delimiterEvaluator, final int evaluationType) { + this.subjectEvaluator = subjectEvaluator; + this.delimiterEvaluator = delimiterEvaluator; + this.evaluationType = evaluationType; + } + + @Override + public QueryResult evaluate(final Map attributes) { + if (delineatedValues == null) { + final QueryResult subjectValue = subjectEvaluator.evaluate(attributes); + if (subjectValue.getValue() == null) { + evaluationsLeft = 0; + return new StringQueryResult(null); + } + + final QueryResult delimiterValue = delimiterEvaluator.evaluate(attributes); + if (subjectValue.getValue() == null) { + evaluationsLeft = 0; + return new StringQueryResult(null); + } + + delineatedValues = subjectValue.getValue().split(delimiterValue.getValue()); + } + + if (evaluationCount > delineatedValues.length) { + evaluationsLeft = 0; + return new StringQueryResult(null); + } + + evaluationsLeft = delineatedValues.length - evaluationCount - 1; + + return new StringQueryResult(delineatedValues[evaluationCount++]); + } + + @Override + public int getEvaluationsRemaining() { + return evaluationsLeft; + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + + @Override + public int getEvaluationType() { + return evaluationType; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java new file mode 100644 index 0000000000..f80ed972b9 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiAttributeEvaluator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.selection; + +import org.apache.nifi.attribute.expression.language.evaluation.StringEvaluator; + +public abstract class MultiAttributeEvaluator extends StringEvaluator { + + public abstract int getEvaluationType(); +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java new file mode 100644 index 0000000000..9a441ce35b --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiMatchAttributeEvaluator.java @@ -0,0 +1,82 @@ +/* + * 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.attribute.expression.language.evaluation.selection; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class MultiMatchAttributeEvaluator extends MultiAttributeEvaluator { + + private final List attributePatterns; + private final int evaluationType; + private final List attributeNames = new ArrayList<>(); + private int evaluationCount = 0; + + public MultiMatchAttributeEvaluator(final List attributeRegexes, final int evaluationType) { + this.attributePatterns = new ArrayList<>(); + for (final String regex : attributeRegexes) { + attributePatterns.add(Pattern.compile(regex)); + } + + this.evaluationType = evaluationType; + } + + /** + * Can be called only after the first call to evaluate + * + * @return + */ + @Override + public int getEvaluationsRemaining() { + return attributeNames.size() - evaluationCount; + } + + @Override + public QueryResult evaluate(final Map attributes) { + if (evaluationCount == 0) { + for (final Pattern pattern : attributePatterns) { + for (final String attrName : attributes.keySet()) { + if (pattern.matcher(attrName).matches()) { + attributeNames.add(attrName); + } + } + } + } + + if (evaluationCount >= attributeNames.size()) { + return new StringQueryResult(null); + } + + return new StringQueryResult(attributes.get(attributeNames.get(evaluationCount++))); + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + + @Override + public int getEvaluationType() { + return evaluationType; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java new file mode 100644 index 0000000000..6dabc0abe3 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/evaluation/selection/MultiNamedAttributeEvaluator.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.evaluation.selection; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.evaluation.StringQueryResult; + +public class MultiNamedAttributeEvaluator extends MultiAttributeEvaluator { + + private final List attributeNames; + private final int evaluationType; + private int evaluationCount = 0; + private List matchingAttributeNames = null; + + public MultiNamedAttributeEvaluator(final List attributeNames, final int evaluationType) { + this.attributeNames = attributeNames; + this.evaluationType = evaluationType; + } + + @Override + public QueryResult evaluate(final Map attributes) { + matchingAttributeNames = new ArrayList<>(attributeNames); + + if (matchingAttributeNames.size() <= evaluationCount) { + return new StringQueryResult(null); + } + + return new StringQueryResult(attributes.get(matchingAttributeNames.get(evaluationCount++))); + } + + @Override + public int getEvaluationsRemaining() { + return matchingAttributeNames.size() - evaluationCount; + } + + @Override + public Evaluator getSubjectEvaluator() { + return null; + } + + @Override + public int getEvaluationType() { + return evaluationType; + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java new file mode 100644 index 0000000000..47d42cb7ff --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageException.java @@ -0,0 +1,34 @@ +/* + * 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.attribute.expression.language.exception; + +public class AttributeExpressionLanguageException extends RuntimeException { + + private static final long serialVersionUID = -5637284498692447901L; + + public AttributeExpressionLanguageException(final String explanation) { + super(explanation); + } + + public AttributeExpressionLanguageException(final String explanation, final Throwable t) { + super(explanation, t); + } + + public AttributeExpressionLanguageException(final Throwable t) { + super(t); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java new file mode 100644 index 0000000000..f8531cb052 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/AttributeExpressionLanguageParsingException.java @@ -0,0 +1,34 @@ +/* + * 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.attribute.expression.language.exception; + +public class AttributeExpressionLanguageParsingException extends AttributeExpressionLanguageException { + + private static final long serialVersionUID = 7422163230677064726L; + + public AttributeExpressionLanguageParsingException(final String explanation) { + super(explanation); + } + + public AttributeExpressionLanguageParsingException(final String explanation, final Throwable t) { + super(explanation, t); + } + + public AttributeExpressionLanguageParsingException(final Throwable t) { + super(t); + } +} diff --git a/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java new file mode 100644 index 0000000000..4a9a9c50b9 --- /dev/null +++ b/commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/exception/IllegalAttributeException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.attribute.expression.language.exception; + +public class IllegalAttributeException extends RuntimeException { + + public IllegalAttributeException() { + super(); + } + + public IllegalAttributeException(final String explanation) { + super(explanation); + } +} diff --git a/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java new file mode 100644 index 0000000000..a2b72141bb --- /dev/null +++ b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestQuery.java @@ -0,0 +1,1068 @@ +/* + * 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.attribute.expression.language; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.text.SimpleDateFormat; +import java.util.Calendar; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.Query.Range; +import org.apache.nifi.attribute.expression.language.evaluation.QueryResult; +import org.apache.nifi.attribute.expression.language.exception.AttributeExpressionLanguageParsingException; +import org.apache.nifi.expression.AttributeExpression.ResultType; +import org.apache.nifi.flowfile.FlowFile; + +import org.antlr.runtime.tree.Tree; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; +import org.mockito.Mockito; + +public class TestQuery { + + @Test + public void testCompilation() { + assertInvalid("${attr:uuid()}"); + assertInvalid("${attr:indexOf(length())}"); + assertValid("${UUID()}"); + assertInvalid("${UUID():nextInt()}"); + assertValid("${nextInt()}"); + assertValid("${now():format('yyyy/MM/dd')}"); + assertInvalid("${attr:times(3)}"); + assertValid("${attr:toNumber():multiply(3)}"); + // left here because it's convenient for looking at the output + //System.out.println(Query.compile("").evaluate(null)); + } + + private void assertValid(final String query) { + try { + Query.compile(query); + } catch (final Exception e) { + e.printStackTrace(); + Assert.fail("Expected query to be valid, but it failed to compile due to " + e); + } + } + + private void assertInvalid(final String query) { + try { + Query.compile(query); + Assert.fail("Expected query to be invalid, but it did compile"); + } catch (final Exception e) { + } + } + + @Test + public void testIsValidExpression() { + Query.validateExpression("${abc:substring(${xyz:length()})}", false); + Query.isValidExpression("${now():format('yyyy-MM-dd')}"); + + + try { + Query.validateExpression("$${attr}", false); + Assert.fail("invalid query validated"); + } catch (final AttributeExpressionLanguageParsingException e) { + } + + Query.validateExpression("$${attr}", true); + + Query.validateExpression("${filename:startsWith('T8MTXBC')\n" + + ":or( ${filename:startsWith('C4QXABC')} )\n" + + ":or( ${filename:startsWith('U6CXEBC')} )" + + ":or( ${filename:startsWith('KYM3ABC')} )}", false); + } + + + @Test + public void testCompileEmbedded() { + final String expression = "${x:equals( ${y} )}"; + final Query query = Query.compile(expression); + final Tree tree = query.getTree(); + System.out.println( printTree(tree) ); + + final Map attributes = new HashMap<>(); + attributes.put("x", "x"); + attributes.put("y", "x"); + final String result = Query.evaluateExpressions(expression, attributes, null); + assertEquals("true", result); + + Query.validateExpression(expression, false); + } + + private String printTree(final Tree tree) { + final StringBuilder sb = new StringBuilder(); + printTree(tree, 0, sb); + + return sb.toString(); + } + + private void printTree(final Tree tree, final int spaces, final StringBuilder sb) { + for (int i=0; i < spaces; i++) { + sb.append(" "); + } + + if ( tree.getText().trim().isEmpty() ) { + sb.append(tree.toString()).append("\n"); + } else { + sb.append(tree.getText()).append("\n"); + } + + for (int i=0; i < tree.getChildCount(); i++) { + printTree(tree.getChild(i), spaces + 2, sb); + } + } + + @Test + public void testEscape() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "My Value"); + attributes.put("${xx}", "hello"); + + assertEquals("My Value", evaluateQueryForEscape("${attr}", attributes)); + assertEquals("${attr}", evaluateQueryForEscape("$${attr}", attributes)); + assertEquals("$My Value", evaluateQueryForEscape("$$${attr}", attributes)); + assertEquals("$${attr}", evaluateQueryForEscape("$$$${attr}", attributes)); + assertEquals("$$My Value", evaluateQueryForEscape("$$$$${attr}", attributes)); + } + + @Test + public void testWithBackSlashes() { + final Map attributes = new HashMap<>(); + attributes.put("x", "C:\\test\\1.txt"); + attributes.put("y", "y\ny"); + + final String query = "${x:substringAfterLast( '/' ):substringAfterLast( '\\\\' )}"; + verifyEquals(query, attributes, "1.txt"); + attributes.put("x", "C:/test/1.txt"); + verifyEquals(query, attributes, "1.txt"); + + verifyEquals("${y:equals('y\\ny')}", attributes, Boolean.TRUE); + } + + @Test + public void testWithTicksOutside() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "My Value"); + + assertEquals(1, Query.extractExpressionRanges("\"${attr}").size()); + assertEquals(1, Query.extractExpressionRanges("'${attr}").size()); + assertEquals(1, Query.extractExpressionRanges("'${attr}'").size()); + assertEquals(1, Query.extractExpressionRanges("${attr}").size()); + + assertEquals("'My Value'", Query.evaluateExpressions("'${attr}'", attributes, null)); + assertEquals("'My Value", Query.evaluateExpressions("'${attr}", attributes, null)); + } + + + @Test + @Ignore("Depends on TimeZone") + public void testDateToNumber() { + final Query query = Query.compile("${dateTime:toDate('yyyy/MM/dd HH:mm:ss.SSS'):toNumber()}"); + final Map attributes = new HashMap<>(); + attributes.put("dateTime", "2013/11/18 10:22:27.678"); + + final QueryResult result = query.evaluate(attributes); + assertEquals(ResultType.NUMBER, result.getResultType()); + assertEquals(1384788147678L, result.getValue()); + } + + @Test + public void testAddOneDayToDate() { + final Map attributes = new HashMap<>(); + attributes.put("dateTime", "2013/11/18 10:22:27.678"); + + verifyEquals("${dateTime:toDate('yyyy/MM/dd HH:mm:ss.SSS'):toNumber():plus(86400000):toDate():format('yyyy/MM/dd HH:mm:ss.SSS')}", attributes, "2013/11/19 10:22:27.678"); + } + + @Test + public void implicitDateConversion() { + final Date date = new Date(); + final Query query = Query.compile("${dateTime:format('yyyy/MM/dd HH:mm:ss.SSS')}"); + final Map attributes = new HashMap<>(); + attributes.put("dateTime", date.toString()); + + // the date.toString() above will end up truncating the milliseconds. So remove millis from the Date before + // formatting it + final SimpleDateFormat sdf = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS"); + final long millis = date.getTime() % 1000L; + final Date roundedToNearestSecond = new Date(date.getTime() - millis); + final String formatted = sdf.format(roundedToNearestSecond); + + final QueryResult result = query.evaluate(attributes); + assertEquals(ResultType.STRING, result.getResultType()); + assertEquals(formatted, result.getValue()); + } + + + @Test + public void testEmbeddedExpressionsAndQuotes() { + final Map attributes = new HashMap<>(); + attributes.put("x", "abc"); + attributes.put("a", "abc"); + + verifyEquals("${x:equals(${a})}", attributes, true); + + Query.validateExpression("${x:equals('${a}')}", false); + assertEquals("true", Query.evaluateExpressions("${x:equals('${a}')}", attributes, null)); + + Query.validateExpression("${x:equals(\"${a}\")}", false); + assertEquals("true", Query.evaluateExpressions("${x:equals(\"${a}\")}", attributes, null)); + } + + + @Test + public void testCurlyBracesInQuotes() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "My Valuee"); + + assertEquals("Val", evaluateQueryForEscape("${attr:replaceAll('My (Val)ue{1,2}', '$1')}", attributes)); + assertEquals("Val", evaluateQueryForEscape("${attr:replaceAll(\"My (Val)ue{1,2}\", '$1')}", attributes)); + } + + + private String evaluateQueryForEscape(final String queryString, final Map attributes) { + FlowFile mockFlowFile = Mockito.mock(FlowFile.class); + Mockito.when(mockFlowFile.getAttributes()).thenReturn(attributes); + Mockito.when(mockFlowFile.getId()).thenReturn(1L); + Mockito.when(mockFlowFile.getEntryDate()).thenReturn(System.currentTimeMillis()); + Mockito.when(mockFlowFile.getSize()).thenReturn(1L); + Mockito.when(mockFlowFile.getLineageIdentifiers()).thenReturn(new HashSet()); + Mockito.when(mockFlowFile.getLineageStartDate()).thenReturn(System.currentTimeMillis()); + return Query.evaluateExpressions(queryString, mockFlowFile); + } + + + @Test + public void testGetAttributeValue() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "My Value"); + verifyEquals("${attr}", attributes, "My Value"); + } + + @Test + public void testGetAttributeValueEmbedded() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "XX "); + attributes.put("XX", "My Value"); + verifyEquals("${${attr:trim()}}", attributes, "My Value"); + } + + @Test + public void testSimpleSubstring() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "My Value"); + verifyEquals("${attr:substring(2, 5)}", attributes, " Va"); + } + + @Test + public void testCallToFunctionWithSubjectResultOfAnotherFunctionCall() { + final Map attributes = new HashMap<>(); + attributes.put("attr", " My Value "); + verifyEquals("${attr:trim():substring(2, 5)}", attributes, " Va"); + } + + @Test + public void testProblematic1() { + // There was a bug that prevented this expression from compiling. This test just verifies that it now compiles. + final String queryString = "${xx:append( \"120101\" ):toDate( 'yyMMddHHmmss' ):format( \"yy-MM-dd’T’HH:mm:ss\") }"; + Query.compile(queryString); + } + + @Test + public void testEquals() { + final Map attributes = new HashMap<>(); + attributes.put("attr", " XX "); + verifyEquals("${attr:trim():equals('XX')}", attributes, true); + } + + @Test + public void testDeeplyEmbedded() { + final Map attributes = new HashMap<>(); + attributes.put("x", "false"); + attributes.put("abc", "a"); + attributes.put("a", "a"); + + verifyEquals("${x:or( ${${abc}:length():equals(1)} )}", attributes, true); + } + + + @Test + public void testExtractExpressionRanges() { + List ranges = Query.extractExpressionRanges("hello"); + assertTrue(ranges.isEmpty()); + + ranges = Query.extractExpressionRanges("${hello"); + assertTrue(ranges.isEmpty()); + + ranges = Query.extractExpressionRanges("hello}"); + assertTrue(ranges.isEmpty()); + + ranges = Query.extractExpressionRanges("$${hello"); + assertTrue(ranges.isEmpty()); + + ranges = Query.extractExpressionRanges("$he{ll}o"); + assertTrue(ranges.isEmpty()); + + ranges = Query.extractExpressionRanges("${hello}"); + assertEquals(1, ranges.size()); + Range range = ranges.get(0); + assertEquals(0, range.getStart()); + assertEquals(7, range.getEnd()); + + ranges = Query.extractExpressionRanges("${hello:equals( ${goodbye} )}"); + assertEquals(1, ranges.size()); + range = ranges.get(0); + assertEquals(0, range.getStart()); + assertEquals(28, range.getEnd()); + + ranges = Query.extractExpressionRanges("${hello:equals( $${goodbye} )}"); + assertEquals(1, ranges.size()); + range = ranges.get(0); + assertEquals(0, range.getStart()); + assertEquals(29, range.getEnd()); + + ranges = Query.extractExpressionRanges("${hello:equals( $${goodbye} )} or just hi, ${bob:or(${jerry})}"); + assertEquals(2, ranges.size()); + range = ranges.get(0); + assertEquals(0, range.getStart()); + assertEquals(29, range.getEnd()); + + range = ranges.get(1); + assertEquals(43, range.getStart()); + assertEquals(61, range.getEnd()); + + + ranges = Query.extractExpressionRanges("${hello:equals( ${goodbye} )} or just hi, ${bob}, are you ${bob.age:toNumber()} yet? $$$${bob}"); + assertEquals(3, ranges.size()); + range = ranges.get(0); + assertEquals(0, range.getStart()); + assertEquals(28, range.getEnd()); + + range = ranges.get(1); + assertEquals(42, range.getStart()); + assertEquals(47, range.getEnd()); + + range = ranges.get(2); + assertEquals(58, range.getStart()); + assertEquals(78, range.getEnd()); + + ranges = Query.extractExpressionRanges("${x:matches( '.{4}' )}"); + assertEquals(1, ranges.size()); + range = ranges.get(0); + assertEquals(0, range.getStart()); + assertEquals(21, range.getEnd()); + } + + + @Test + public void testExtractExpressionTypes() { + List types = Query.extractResultTypes("${hello:equals( ${goodbye} )} or just hi, ${bob}, are you ${bob.age:toNumber()} yet? $$$${bob}"); + assertEquals(3, types.size()); + assertEquals(ResultType.BOOLEAN, types.get(0)); + assertEquals(ResultType.STRING, types.get(1)); + assertEquals(ResultType.NUMBER, types.get(2)); + } + + + @Test + public void testEqualsEmbedded() { + final Map attributes = new HashMap<>(); + attributes.put("x", "hello"); + attributes.put("y", "good-bye"); + + verifyEquals("${x:equals( ${y} )}", attributes, false); + + attributes.put("y", "hello"); + verifyEquals("${x:equals( ${y} )}", attributes, true); + + attributes.put("x", "4"); + attributes.put("y", "3"); + attributes.put("z", "1"); + attributes.put("h", "100"); + verifyEquals("${x:toNumber():lt( ${y:toNumber():plus( ${h:toNumber()} )} )}", attributes, true); + verifyEquals("${h:toNumber():ge( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true); + verifyEquals("${x:toNumber():equals( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true); + + attributes.put("x", "88"); + verifyEquals("${x:toNumber():gt( ${y:toNumber():plus( ${z:toNumber()} )} )}", attributes, true); + + attributes.put("y", "88"); + assertEquals("true", Query.evaluateExpressions("${x:equals( '${y}' )}", attributes, null)); + } + + + @Test + public void testComplicatedEmbeddedExpressions() { + final Map attributes = new HashMap<>(); + attributes.put("fox", "quick, brown"); + attributes.put("dog", "lazy"); + + verifyEquals("${fox:substring( ${ 'dog' :substring(2):length()}, 5 ):equals( 'ick' )}", attributes, true); + verifyEquals("${fox:substring( ${ 'dog' :substring(2):length()}, 5 ):equals( 'ick' )}", attributes, true); + } + + @Test + public void testQuotingQuotes() { + final Map attributes = new HashMap<>(); + attributes.put("xx", "say 'hi'"); + + String query = "${xx:replaceAll( \"'.*'\", '\\\"hello\\\"' )}"; + verifyEquals(query, attributes, "say \"hello\""); + + query = "${xx:replace( \"'\", '\"')}"; + verifyEquals(query, attributes, "say \"hi\""); + + query = "${xx:replace( '\\'', '\"')}"; + System.out.println(query); + verifyEquals(query, attributes, "say \"hi\""); + } + + @Test + public void testDoubleQuotesWithinSingleQuotes() { + final Map attributes = new HashMap<>(); + attributes.put("xx", "say 'hi'"); + + String query = "${xx:replace( \"'hi'\", '\\\"hello\\\"' )}"; + System.out.println(query); + verifyEquals(query, attributes, "say \"hello\""); + } + + @Test + public void testEscapeQuotes() { + final long timestamp = 1403620278642L; + final Map attributes = new HashMap<>(); + attributes.put("date", String.valueOf(timestamp)); + + final String format = "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'"; + + final String query = "startDateTime=\"${date:toNumber():toDate():format(\"" + format + "\")}\""; + final String result = Query.evaluateExpressions(query, attributes, null); + + final String expectedTime = new SimpleDateFormat(format).format(timestamp); + assertEquals("startDateTime=\"" + expectedTime + "\"", result); + + final List ranges = Query.extractExpressionRanges(query); + assertEquals(1, ranges.size()); + } + + @Test + public void testDateConversion() { + final Map attributes = new HashMap<>(); + attributes.put("date", "1403620278642"); + + verifyEquals("${date:format('yyyy')}", attributes, "2014"); + verifyEquals("${date:toDate():format('yyyy')}", attributes, "2014"); + verifyEquals("${date:toNumber():format('yyyy')}", attributes, "2014"); + verifyEquals("${date:toNumber():toDate():format('yyyy')}", attributes, "2014"); + verifyEquals("${date:toDate():toNumber():format('yyyy')}", attributes, "2014"); + verifyEquals("${date:toDate():toNumber():toDate():toNumber():toDate():toNumber():format('yyyy')}", attributes, "2014"); + } + + @Test + public void testSingleLetterAttribute() { + final Map attributes = new HashMap<>(); + attributes.put("A", "0123456789"); + + verifyEquals("${A}", attributes, "0123456789"); + verifyEquals("${'A'}", attributes, "0123456789"); + } + + + @Test + public void testImplicitConversions() { + final Map attributes = new HashMap<>(); + attributes.put("A", "0123456789"); + attributes.put("b", "true"); + attributes.put("c", "false"); + attributes.put("d", "Quick Brown Fox"); + attributes.put("F", "-48"); + attributes.put("n", "2014/04/04 00:00:00"); + + final Calendar cal = Calendar.getInstance(); + cal.set(Calendar.YEAR, 2014); + cal.set(Calendar.MONTH, 3); + cal.set(Calendar.DAY_OF_MONTH, 4); + cal.set(Calendar.HOUR, 0); + cal.set(Calendar.MINUTE, 0); + cal.set(Calendar.SECOND, 45); + + final String dateString = cal.getTime().toString(); + attributes.put("z", dateString); + + + verifyEquals("${A:plus(4)}", attributes, 123456793L); + verifyEquals("${A:plus( ${F} )}", attributes, 123456741L); + + verifyEquals("${F:lt( ${A} )}", attributes, true); + verifyEquals("${A:substring(2,3):plus(21):substring(1,2):plus(0)}", attributes, 3L); + verifyEquals("${n:format( 'yyyy' )}", attributes, "2014"); + verifyEquals("${z:format( 'yyyy' )}", attributes, "2014"); + + attributes.put("n", "2014/04/04 00:00:00.045"); + verifyEquals("${n:format( 'yyyy' ):append(','):append( ${n:format( 'SSS' )} )}", attributes, "2014,045"); + } + + @Test + public void testNewLinesAndTabsInQuery() { + final String query = "${ abc:equals('abc'):or( \n\t${xx:isNull()}\n) }"; + assertEquals(ResultType.BOOLEAN, Query.getResultType(query)); + Query.validateExpression(query, false); + assertEquals("true", Query.evaluateExpressions(query)); + } + + @Test + public void testAttributeReferencesWithWhiteSpace() { + final Map attrs = new HashMap<>(); + attrs.put("a b c,d", "abc"); + + final String query = "${ 'a b c,d':equals('abc') }"; + verifyEquals(query, attrs, true); + } + + @Test + public void testComments() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "xyz"); + + final String expression = + "# hello, world\n" + + "${# ref attr\n" + + "\t" + + "abc" + + "\t" + + "#end ref attr\n" + + "}"; + + Query query = Query.compile(expression); + QueryResult result = query.evaluate(attributes); + assertEquals(ResultType.STRING, result.getResultType()); + assertEquals("xyz", result.getValue()); + + query = Query.compile("${abc:append('# hello') #good-bye \n}"); + result = query.evaluate(attributes); + assertEquals(ResultType.STRING, result.getResultType()); + assertEquals("xyz# hello", result.getValue()); + } + + @Test + public void testAppendPrepend() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "XX"); + attributes.put("YXXX", "bingo"); + + verifyEquals("${${attr:append('X'):prepend('Y')}}", attributes, "bingo"); + } + + @Test + public void testIsNull() { + final Map attributes = new HashMap<>(); + verifyEquals("${attr:isNull()}", attributes, true); + } + + @Test + public void testNotNull() { + final Map attributes = new HashMap<>(); + attributes.put("attr", ""); + + verifyEquals("${attr:notNull()}", attributes, true); + } + + @Test + public void testIsNullOrLengthEquals0() { + final Map attributes = new HashMap<>(); + attributes.put("abc", ""); + attributes.put("xyz", "xyz"); + attributes.put("xx", " "); + + verifyEquals("${abc:isNull():or( ${abc:length():equals(0)} )}", attributes, true); + verifyEquals("${xyz:isNull():or( ${xyz:length():equals(0)} )}", attributes, false); + verifyEquals("${none:isNull():or( ${none:length():equals(0)} )}", attributes, true); + verifyEquals("${xx:isNull():or( ${xx:trim():length():equals(0)} )}", attributes, true); + } + + @Test + public void testReplaceNull() { + final Map attributes = new HashMap<>(); + verifyEquals("${attr:replaceNull('hello')}", attributes, "hello"); + } + + @Test + public void testReplace() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "hello"); + verifyEquals("${attr:replace('hell', 'yell')}", attributes, "yello"); + } + + @Test + public void testReplaceAll() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "hello"); + attributes.put("xyz", "00-00TEST.2014_01_01_000000_value"); + + verifyEquals("${xyz:replaceAll(\"^([^.]+)\\.([0-9]{4})_([0-9]{2})_([0-9]{2}).*$\", \"$3\")}", attributes, "01"); + verifyEquals("${attr:replaceAll('l+', 'r')}", attributes, "hero"); + + attributes.clear(); + attributes.put("filename1", "abc.gz"); + attributes.put("filename2", "abc.g"); + attributes.put("filename3", "abc.gz.gz"); + attributes.put("filename4", "abc.gz.g"); + attributes.put("abc", "hello world"); + + verifyEquals("${filename3:replaceAll('\\\\\\.gz$', '')}", attributes, "abc.gz.gz"); + verifyEquals("${filename3:replaceAll('\\\\\\\\.gz$', '')}", attributes, "abc.gz.gz"); + verifyEquals("${filename1:replaceAll('\\.gz$', '')}", attributes, "abc"); + verifyEquals("${filename2:replaceAll('\\.gz$', '')}", attributes, "abc.g"); + verifyEquals("${filename4:replaceAll('\\\\.gz$', '')}", attributes, "abc.gz.g"); + + verifyEquals("${abc:replaceAll( 'lo wor(ld)', '$0')}", attributes, "hello world"); + verifyEquals("${abc:replaceAll( 'he(llo) world', '$1')}", attributes, "llo"); + verifyEquals("${abc:replaceAll( 'xx', '$0')}", attributes, "hello world"); + verifyEquals("${abc:replaceAll( '(xx)', '$1')}", attributes, "hello world"); + verifyEquals("${abc:replaceAll( 'lo wor(ld)', '$1')}", attributes, "helld"); + + } + + + @Test + public void testReplaceAllWithOddNumberOfBackslashPairs() { + final Map attributes = new HashMap<>(); + attributes.put("filename", "C:\\temp\\.txt"); + + verifyEquals("${filename:replace('\\\\', '/')}", attributes, "C:/temp/.txt"); + verifyEquals("${filename:replaceAll('\\\\\\\\', '/')}", attributes, "C:/temp/.txt"); + verifyEquals("${filename:replaceAll('\\\\\\.txt$', '')}", attributes, "C:\\temp"); + } + + @Test + public void testReplaceAllWithMatchingGroup() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "hello"); + + verifyEquals("${attr:replaceAll('.*?(l+).*', '$1')}", attributes, "ll"); + } + + @Test + public void testMathOperations() { + final Map attributes = new HashMap<>(); + attributes.put("one", "1"); + attributes.put("two", "2"); + attributes.put("three", "3"); + attributes.put("four", "4"); + attributes.put("five", "5"); + attributes.put("hundred", "100"); + + verifyEquals("${hundred:toNumber():multiply(2):divide(3):plus(1):mod(5)}", attributes, 2L); + } + + @Test + public void testIndexOf() { + final Map attributes = new HashMap<>(); + attributes.put("attr", "https://abc.go"); + verifyEquals("${attr:indexOf('/')}", attributes, 6L); + } + + @Test + public void testDate() { + final Calendar now = Calendar.getInstance(); + final int year = now.get(Calendar.YEAR); + final Map attributes = new HashMap<>(); + attributes.put("entryDate", String.valueOf(now.getTimeInMillis())); + + verifyEquals("${entryDate:toNumber():toDate():format('yyyy')}", attributes, String.valueOf(year)); + + attributes.clear(); + attributes.put("month", "3"); + attributes.put("day", "4"); + attributes.put("year", "2013"); + assertEquals("63", Query.evaluateExpressions("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, null)); + assertEquals("63", Query.evaluateExpressions("${year:append('/'):append('${month}'):append('/'):append('${day}'):toDate('yyyy/MM/dd'):format('D')}", attributes, null)); + + verifyEquals("${year:append('/'):append(${month}):append('/'):append(${day}):toDate('yyyy/MM/dd'):format('D')}", attributes, "63"); + } + + @Test + public void testSystemProperty() { + System.setProperty("hello", "good-bye"); + assertEquals("good-bye", Query.evaluateExpressions("${hello}")); + assertEquals("good-bye", Query.compile("${hello}").evaluate().getValue()); + } + + @Test + public void testAnyAttribute() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "zzz"); + attributes.put("xyz", "abc"); + + verifyEquals("${anyAttribute('abc', 'xyz', 'missingAttr'):substring(1,2):equals('b')}", attributes, true); + verifyEquals("${anyAttribute('abc', 'xyz'):substring(1,2):equals('b')}", attributes, true); + verifyEquals("${anyAttribute('xyz', 'abc'):substring(1,2):equals('b')}", attributes, true); + verifyEquals("${anyAttribute('zz'):substring(1,2):equals('b')}", attributes, false); + verifyEquals("${anyAttribute('abc', 'zz'):isNull()}", attributes, true); + } + + @Test + public void testAnyMatchingAttribute() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "zzz"); + attributes.put("xyz", "abc"); + attributes.put("123.cba", "hello"); + + verifyEquals("${anyMatchingAttribute('.{2}x', '.{2}z'):substring(1,2):equals('b')}", attributes, true); + verifyEquals("${anyMatchingAttribute('.*'):substring(1,2):equals('b')}", attributes, true); + verifyEquals("${anyMatchingAttribute('x{44}'):substring(1,2):equals('b')}", attributes, false); + verifyEquals("${anyMatchingAttribute('abc'):substring(1,2):equals('b')}", attributes, false); + verifyEquals("${anyMatchingAttribute('xyz'):substring(1,2):equals('b')}", attributes, true); + verifyEquals("${anyMatchingAttribute('xyz'):notNull()}", attributes, true); + verifyEquals("${anyMatchingAttribute('xyz'):isNull()}", attributes, false); + verifyEquals("${anyMatchingAttribute('xxxxxxxxx'):notNull()}", attributes, false); + verifyEquals("${anyMatchingAttribute('123\\.c.*'):matches('hello')}", attributes, true); + verifyEquals("${anyMatchingAttribute('123\\.c.*|a.c'):matches('zzz')}", attributes, true); + } + + + @Test + public void testAnyDelineatedValue() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "a,b,c"); + attributes.put("xyz", "abc"); + + final String query = "${anyDelineatedValue('${abc}', ','):equals('b')}"; + assertEquals(ResultType.BOOLEAN, Query.getResultType(query)); + + assertEquals("true", Query.evaluateExpressions(query, attributes, null)); + assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('a')}", attributes, null)); + assertEquals("true", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('c')}", attributes, null)); + assertEquals("false", Query.evaluateExpressions("${anyDelineatedValue('${abc}', ','):equals('d')}", attributes, null)); + + verifyEquals("${anyDelineatedValue(${abc}, ','):equals('b')}", attributes, true); + verifyEquals("${anyDelineatedValue(${abc}, ','):equals('a')}", attributes, true); + verifyEquals("${anyDelineatedValue(${abc}, ','):equals('c')}", attributes, true); + verifyEquals("${anyDelineatedValue(${abc}, ','):equals('d')}", attributes, false); + } + + @Test + public void testAllDelineatedValues() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "a,b,c"); + attributes.put("xyz", "abc"); + + final String query = "${allDelineatedValues('${abc}', ','):matches('[abc]')}"; + + assertEquals(ResultType.BOOLEAN, Query.getResultType(query)); + assertEquals("true", Query.evaluateExpressions(query, attributes, null)); + assertEquals("true", Query.evaluateExpressions(query, attributes, null)); + assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):matches('[abd]')}", attributes, null)); + assertEquals("false", Query.evaluateExpressions("${allDelineatedValues('${abc}', ','):equals('a'):not()}", attributes, null)); + + verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abc]')}", attributes, true); + verifyEquals("${allDelineatedValues(${abc}, ','):matches('[abd]')}", attributes, false); + verifyEquals("${allDelineatedValues(${abc}, ','):equals('a'):not()}", attributes, false); + } + + + @Test + public void testAllAttributes() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "1234"); + attributes.put("xyz", "4132"); + attributes.put("hello", "world!"); + + verifyEquals("${allAttributes('abc', 'xyz'):matches('\\d+')}", attributes, true); + verifyEquals("${allAttributes('abc', 'xyz'):toNumber():lt(99999)}", attributes, true); + verifyEquals("${allAttributes('abc', 'hello'):length():gt(3)}", attributes, true); + verifyEquals("${allAttributes('abc', 'hello'):length():equals(4)}", attributes, false); + verifyEquals("${allAttributes('abc', 'xyz'):length():equals(4)}", attributes, true); + verifyEquals("${allAttributes('abc', 'xyz', 'other'):isNull()}", attributes, false); + + try { + Query.compile("${allAttributes('#ah'):equals('hello')"); + Assert.fail("Was able to compile with allAttributes and an invalid attribute name"); + } catch (final AttributeExpressionLanguageParsingException e) { + // expected behavior + } + } + + + @Test + public void testMathOperators() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "1234"); + attributes.put("xyz", "4132"); + attributes.put("hello", "world!"); + + verifyEquals("${xyz:toNumber():gt( ${abc:toNumber()} )}", attributes, true); + } + + @Test + public void testAllMatchingAttributes() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "1234"); + attributes.put("xyz", "4132"); + attributes.put("hello", "world!"); + attributes.put("123.cba", "hell.o"); + + System.out.println( printTree(Query.compile("${allMatchingAttributes('(abc|xyz)'):matches('\\\\d+')}").getTree()) ); + + verifyEquals("${'123.cba':matches('hell\\.o')}", attributes, true); + verifyEquals("${allMatchingAttributes('123\\.cba'):equals('hell.o')}", attributes, true); + verifyEquals("${allMatchingAttributes('(abc|xyz)'):matches('\\d+')}", attributes, true); + verifyEquals("${allMatchingAttributes('[ax].*'):toNumber():lt(99999)}", attributes, true); + verifyEquals("${allMatchingAttributes('hell.'):length():gt(3)}", attributes, true); + + verifyEquals("${allMatchingAttributes('123\\.cba'):equals('no')}", attributes, false); + } + + @Test + public void testMatches() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "1234xyz4321"); + attributes.put("end", "xyz"); + attributes.put("xyz", "4132"); + attributes.put("hello", "world!"); + attributes.put("dotted", "abc.xyz"); + + final String evaluated = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null); + assertEquals("true", evaluated); + + attributes.put("end", "888"); + final String secondEvaluation = Query.evaluateExpressions("${abc:matches('1234${end}4321')}", attributes, null); + assertEquals("false", secondEvaluation); + + verifyEquals("${dotted:matches('abc\\.xyz')}", attributes, true); + } + + + @Test + public void testFind() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "1234xyz4321"); + attributes.put("end", "xyz"); + attributes.put("xyz", "4132"); + attributes.put("hello", "world!"); + attributes.put("dotted", "abc.xyz"); + + final String evaluated = Query.evaluateExpressions("${abc:find('1234${end}4321')}", attributes, null); + assertEquals("true", evaluated); + + attributes.put("end", "888"); + final String secondEvaluation = Query.evaluateExpressions("${abc:find('${end}4321')}", attributes, null); + assertEquals("false", secondEvaluation); + + verifyEquals("${dotted:find('\\.')}", attributes, true); + } + + @Test + public void testSubstringAfter() { + final Map attributes = new HashMap<>(); + attributes.put("filename", "file-255"); + + verifyEquals("${filename:substringAfter('')}", attributes, "file-255"); + verifyEquals("${filename:substringAfterLast('')}", attributes, "file-255"); + verifyEquals("${filename:substringBefore('')}", attributes, "file-255"); + verifyEquals("${filename:substringBeforeLast('')}", attributes, "file-255"); + verifyEquals("${filename:substringBefore('file')}", attributes, ""); + + attributes.put("uri", "sftp://some.uri"); + verifyEquals("${uri:substringAfter('sftp')}", attributes, "://some.uri"); + } + + @Test + public void testSubstringAfterLast() { + final Map attributes = new HashMap<>(); + attributes.put("filename", "file-file-255"); + + verifyEquals("${filename:substringAfterLast('file-')}", attributes, "255"); + verifyEquals("${filename:substringAfterLast('5')}", attributes, ""); + verifyEquals("${filename:substringAfterLast('x')}", attributes, "file-file-255"); + } + + @Test + public void testSubstringBefore() { + final Map attributes = new HashMap<>(); + attributes.put("something", "some {} or other"); + + verifyEquals("${something:substringBefore('}')}", attributes, "some {"); + } + + @Test + public void testSubstring() { + final Map attributes = new HashMap<>(); + attributes.put("filename", "file-255"); + + verifyEquals("${filename:substring(1, 2)}", attributes, "i"); + verifyEquals("${filename:substring(4)}", attributes, "-255"); + } + + @Test + public void testToRadix() { + final Map attributes = new HashMap<>(); + attributes.put("filename", "file-255"); + attributes.put("filename2", "file-99999"); + + + verifyEquals("${filename:substringAfter('-'):toNumber():toRadix(16):toUpper()}", attributes, "FF"); + verifyEquals("${filename:substringAfter('-'):toNumber():toRadix(16, 4):toUpper()}", attributes, "00FF"); + verifyEquals("${filename:substringAfter('-'):toNumber():toRadix(36, 3):toUpper()}", attributes, "073"); + } + + @Test + public void testDateFormatConversion() { + final Map attributes = new HashMap<>(); + attributes.put("blue", "20130917162643"); + verifyEquals("${blue:toDate('yyyyMMddHHmmss'):format(\"yyyy/MM/dd HH:mm:ss.SSS'Z'\")}", attributes, "2013/09/17 16:26:43.000Z"); + } + + + @Test + public void testNot() { + verifyEquals("${ab:notNull():not()}", new HashMap(), true); + } + + @Test + public void testAttributesWithSpaces() { + final Map attributes = new HashMap<>(); + attributes.put("ab", "abc"); + attributes.put("a b", "abc"); + + verifyEquals("${ab}", attributes, "abc"); + verifyEquals("${'a b'}", attributes, "abc"); + verifyEquals("${'a b':replaceNull('')}", attributes, ""); + } + + @Test + public void testOr() { + final Map attributes = new HashMap<>(); + attributes.put("filename1", "xabc"); + attributes.put("filename2", "yabc"); + attributes.put("filename3", "abcxy"); + + verifyEquals("${filename1:startsWith('x'):or(true)}", attributes, true); + verifyEquals("${filename1:startsWith('x'):or( ${filename1:startsWith('y')} )}", attributes, true); + verifyEquals("${filename2:startsWith('x'):or( ${filename2:startsWith('y')} )}", attributes, true); + verifyEquals("${filename3:startsWith('x'):or( ${filename3:startsWith('y')} )}", attributes, false); + verifyEquals("${filename1:startsWith('x'):or( ${filename2:startsWith('y')} )}", attributes, true); + verifyEquals("${filename2:startsWith('x'):or( ${filename1:startsWith('y')} )}", attributes, false); + } + + @Test + public void testAnd() { + final Map attributes = new HashMap<>(); + attributes.put("filename1", "xabc"); + attributes.put("filename2", "yabc"); + attributes.put("filename 3", "abcxy"); + + verifyEquals("${filename1:startsWith('x'):and(true)}", attributes, true); + verifyEquals("${filename1:startsWith('x') : and( false )}", attributes, false); + verifyEquals("${filename1:startsWith('x'):and( ${filename1:startsWith('y')} )}", attributes, false); + verifyEquals("${filename2:startsWith('x'):and( ${filename2:startsWith('y')} )}", attributes, false); + verifyEquals("${filename3:startsWith('x'):and( ${filename3:startsWith('y')} )}", attributes, false); + verifyEquals("${filename1:startsWith('x'):and( ${filename2:startsWith('y')} )}", attributes, true); + verifyEquals("${filename2:startsWith('x'):and( ${filename1:startsWith('y')} )}", attributes, false); + verifyEquals("${filename1:startsWith('x'):and( ${'filename 3':endsWith('y')} )}", attributes, true); + } + + @Test + public void testAndOrNot() { + final Map attributes = new HashMap<>(); + attributes.put("filename1", "xabc"); + attributes.put("filename2", "yabc"); + attributes.put("filename 3", "abcxy"); + + final String query = + "${" + + " 'non-existing':notNull():not():and(" + // true AND ( + " ${filename1:startsWith('y')" + // false + " :or(" + // or + " ${ filename1:startsWith('x'):and(false) }" + // false + " ):or(" + // or + " ${ filename2:endsWith('xxxx'):or( ${'filename 3':length():gt(1)} ) }" + // true ) + " )}" + + " )" + + "}"; + + System.out.println(query); + verifyEquals(query, attributes, true); + } + + @Test + public void testAndOrLogicWithAnyAll() { + final Map attributes = new HashMap<>(); + attributes.put("filename1", "xabc"); + attributes.put("filename2", "yabc"); + attributes.put("filename 3", "abcxy"); + + verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):and( ${filename2:equals('yabc')} )}", attributes, true); + verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):and( ${filename2:equals('xabc')} )}", attributes, false); + verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):not():or( ${filename2:equals('yabc')} )}", attributes, true); + verifyEquals("${anyMatchingAttribute('filename.*'):contains('abc'):not():or( ${filename2:equals('xabc')} )}", attributes, false); + } + + @Test + public void testKeywords() { + final Map attributes = new HashMap<>(); + attributes.put("UUID", "123"); + verifyEquals("${ 'UUID':toNumber():equals(123) }", attributes, true); + } + + @Test + public void testEqualsNumber() { + final Map attributes = new HashMap<>(); + attributes.put("abc", "123"); + verifyEquals("${ abc:toNumber():equals(123) }", attributes, true); + } + + @Test + public void testSubjectAsEmbeddedExpressionWithSurroundChars() { + final Map attributes = new HashMap<>(); + attributes.put("b", "x"); + attributes.put("abcxcba", "hello"); + + final String evaluated = Query.evaluateExpressions("${ 'abc${b}cba':substring(0, 1) }", attributes, null); + assertEquals("h", evaluated); + } + + @Test + public void testToNumberFunctionReturnsNumberType() { + assertEquals(ResultType.NUMBER, Query.getResultType("${header.size:toNumber()}")); + } + + + private void verifyEquals(final String expression, final Map attributes, final Object expectedResult) { + Query.validateExpression(expression, false); + assertEquals(String.valueOf(expectedResult), Query.evaluateExpressions(expression, attributes, null)); + + Query query = Query.compile(expression); + QueryResult result = query.evaluate(attributes); + + if ( expectedResult instanceof Number ) { + assertEquals(ResultType.NUMBER, result.getResultType()); + } else if ( expectedResult instanceof Boolean ) { + assertEquals(ResultType.BOOLEAN, result.getResultType()); + } else { + assertEquals(ResultType.STRING, result.getResultType()); + } + + assertEquals(expectedResult, result.getValue()); + } +} diff --git a/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java new file mode 100644 index 0000000000..398a23b640 --- /dev/null +++ b/commons/nifi-expression-language/src/test/java/org/apache/nifi/attribute/expression/language/TestStandardPreparedQuery.java @@ -0,0 +1,92 @@ +/* + * 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.attribute.expression.language; + +import static org.junit.Assert.assertEquals; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.junit.Ignore; +import org.junit.Test; + +public class TestStandardPreparedQuery { + + @Test + public void testSimpleReference() { + final Map attrs = new HashMap<>(); + attrs.put("xx", "world"); + + assertEquals("world", evaluate("${xx}", attrs)); + assertEquals("hello, world!", evaluate("hello, ${xx}!", attrs)); + } + + @Test + public void testEmbeddedReference() { + final Map attrs = new HashMap<>(); + attrs.put("xx", "yy"); + attrs.put("yy", "world"); + + assertEquals("world", evaluate("${${xx}}", attrs)); + } + + @Test + public void test10MIterations() { + final Map attrs = new HashMap<>(); + attrs.put("xx", "world"); + + final StandardPreparedQuery prepared = (StandardPreparedQuery) Query.prepare("${xx}"); + final long start = System.nanoTime(); + for (int i=0; i < 10000000; i++) { + assertEquals( "world", prepared.evaluateExpressions(attrs, null) ); + } + final long nanos = System.nanoTime() - start; + System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos)); + } + + @Test + @Ignore("Takes too long") + public void test10MIterationsWithQuery() { + final Map attrs = new HashMap<>(); + attrs.put("xx", "world"); + + final long start = System.nanoTime(); + for (int i=0; i < 10000000; i++) { + assertEquals( "world", Query.evaluateExpressions("${xx}", attrs) ); + } + final long nanos = System.nanoTime() - start; + System.out.println(TimeUnit.NANOSECONDS.toMillis(nanos)); + + } + + @Test + public void testSeveralSequentialExpressions() { + final Map attributes = new HashMap<>(); + attributes.put("audience", "World"); + attributes.put("comma", ","); + attributes.put("question", " how are you?"); + assertEquals("Hello, World, how are you?!", evaluate("Hello, ${audience}${comma}${question}!", attributes)); + + } + + private String evaluate(final String query, final Map attrs) { + final String evaluated = ((StandardPreparedQuery) Query.prepare(query)).evaluateExpressions(attrs, null); + return evaluated; + } + +} diff --git a/commons/nifi-file-utils/pom.xml b/commons/nifi-file-utils/pom.xml new file mode 100644 index 0000000000..e3cf792a39 --- /dev/null +++ b/commons/nifi-file-utils/pom.xml @@ -0,0 +1,35 @@ + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-file-utils + 0.0.1-SNAPSHOT + jar + NiFi File Utils + + + + commons-codec + commons-codec + 1.10 + + + diff --git a/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java b/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java new file mode 100644 index 0000000000..8920493a16 --- /dev/null +++ b/commons/nifi-file-utils/src/main/java/org/apache/nifi/file/FileUtils.java @@ -0,0 +1,612 @@ +/* + * 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.file; + +import java.io.BufferedInputStream; +import java.io.Closeable; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Random; +import org.apache.commons.codec.digest.DigestUtils; + +import org.slf4j.Logger; + +/** + * A utility class containing a few useful static methods to do typical IO + * operations. + * + * @author unattributed + */ +public class FileUtils { + + public static final long TRANSFER_CHUNK_SIZE_BYTES = 1024 * 1024 * 8; //8 MB chunks + public static final long MILLIS_BETWEEN_ATTEMPTS = 50L; + + /** + * Closes the given closeable quietly - no logging, no exceptions... + * + * @param closeable + */ + public static void closeQuietly(final Closeable closeable) { + if (null != closeable) { + try { + closeable.close(); + } catch (final IOException io) {/*IGNORE*/ + + } + } + } + + /** + * Releases the given lock quietly - no logging, no exception + * + * @param lock + */ + public static void releaseQuietly(final FileLock lock) { + if (null != lock) { + try { + lock.release(); + } catch (final IOException io) { + /*IGNORE*/ + } + } + } + + public static void ensureDirectoryExistAndCanAccess(final File dir) throws IOException { + if (dir.exists() && !dir.isDirectory()) { + throw new IOException(dir.getAbsolutePath() + " is not a directory"); + } else if (!dir.exists()) { + final boolean made = dir.mkdirs(); + if (!made) { + throw new IOException(dir.getAbsolutePath() + " could not be created"); + } + } + if (!(dir.canRead() && dir.canWrite())) { + throw new IOException(dir.getAbsolutePath() + " directory does not have read/write privilege"); + } + } + + /** + * Deletes the given file. If the given file exists but could not be deleted + * this will be printed as a warning to the given logger + * + * @param file + * @param logger + * @return + */ + public static boolean deleteFile(final File file, final Logger logger) { + return FileUtils.deleteFile(file, logger, 1); + } + + /** + * Deletes the given file. If the given file exists but could not be deleted + * this will be printed as a warning to the given logger + * + * @param file + * @param logger + * @param attempts indicates how many times an attempt to delete should be + * made + * @return true if given file no longer exists + */ + public static boolean deleteFile(final File file, final Logger logger, final int attempts) { + if(file == null){ + return false; + } + boolean isGone = false; + try { + if (file.exists()) { + final int effectiveAttempts = Math.max(1, attempts); + for (int i = 0; i < effectiveAttempts && !isGone; i++) { + isGone = file.delete() || !file.exists(); + if (!isGone && (effectiveAttempts - i) > 1) { + FileUtils.sleepQuietly(MILLIS_BETWEEN_ATTEMPTS); + } + } + if (!isGone && logger != null) { + logger.warn("File appears to exist but unable to delete file: " + file.getAbsolutePath()); + } + } + } catch (final Throwable t) { + if (logger != null) { + logger.warn("Unable to delete file: '" + file.getAbsolutePath() + "' due to " + t); + } + } + return isGone; + } + + /** + * Deletes all of the given files. If any exist and cannot be deleted that + * will be printed at warn to the given logger. + * + * @param files can be null + * @param logger can be null + */ + public static void deleteFile(final List files, final Logger logger) { + FileUtils.deleteFile(files, logger, 1); + } + + /** + * Deletes all of the given files. If any exist and cannot be deleted that + * will be printed at warn to the given logger. + * + * @param files can be null + * @param logger can be null + * @param attempts indicates how many times an attempt should be made to + * delete each file + */ + public static void deleteFile(final List files, final Logger logger, final int attempts) { + if (null == files || files.isEmpty()) { + return; + } + final int effectiveAttempts = Math.max(1, attempts); + for (final File file : files) { + try { + boolean isGone = false; + for (int i = 0; i < effectiveAttempts && !isGone; i++) { + isGone = file.delete() || !file.exists(); + if (!isGone && (effectiveAttempts - i) > 1) { + FileUtils.sleepQuietly(MILLIS_BETWEEN_ATTEMPTS); + } + } + if (!isGone && logger != null) { + logger.warn("File appears to exist but unable to delete file: " + file.getAbsolutePath()); + } + } catch (final Throwable t) { + if (null != logger) { + logger.warn("Unable to delete file given from path: '" + file.getPath() + "' due to " + t); + } + } + } + } + + /** + * Deletes all files (not directories..) in the given directory (non + * recursive) that match the given filename filter. If any file cannot be + * deleted then this is printed at warn to the given logger. + * + * @param directory + * @param filter if null then no filter is used + * @param logger + */ + public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger) { + FileUtils.deleteFilesInDir(directory, filter, logger, false); + } + + /** + * Deletes all files (not directories) in the given directory (recursive) + * that match the given filename filter. If any file cannot be deleted then + * this is printed at warn to the given logger. + * + * @param directory + * @param filter if null then no filter is used + * @param logger + * @param recurse + */ + public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse) { + FileUtils.deleteFilesInDir(directory, filter, logger, recurse, false); + } + + /** + * Deletes all files (not directories) in the given directory (recursive) + * that match the given filename filter. If any file cannot be deleted then + * this is printed at warn to the given logger. + * + * @param directory + * @param filter if null then no filter is used + * @param logger + * @param recurse + * @param deleteEmptyDirectories default is false; if true will delete + * directories found that are empty + */ + public static void deleteFilesInDir(final File directory, final FilenameFilter filter, final Logger logger, final boolean recurse, final boolean deleteEmptyDirectories) { + // ensure the specified directory is actually a directory and that it exists + if (null != directory && directory.isDirectory()) { + final File ingestFiles[] = directory.listFiles(); + for (File ingestFile : ingestFiles) { + boolean process = (filter == null) ? true : filter.accept(directory, ingestFile.getName()); + if (ingestFile.isFile() && process) { + FileUtils.deleteFile(ingestFile, logger, 3); + } + if (ingestFile.isDirectory() && recurse) { + FileUtils.deleteFilesInDir(ingestFile, filter, logger, recurse, deleteEmptyDirectories); + if (deleteEmptyDirectories && ingestFile.list().length == 0) { + FileUtils.deleteFile(ingestFile, logger, 3); + } + } + } + } + } + + /** + * Deletes given files. + * + * @param files + * @param recurse will recurse + * @throws IOException + */ + public static void deleteFiles(final Collection files, final boolean recurse) throws IOException { + for (final File file : files) { + FileUtils.deleteFile(file, recurse); + } + } + + public static void deleteFile(final File file, final boolean recurse) throws IOException { + if (file.isDirectory() && recurse) { + FileUtils.deleteFiles(Arrays.asList(file.listFiles()), recurse); + } + //now delete the file itself regardless of whether it is plain file or a directory + if (!FileUtils.deleteFile(file, null, 5)) { + throw new IOException("Unable to delete " + file.getAbsolutePath()); + } + } + + /** + * Randomly generates a sequence of bytes and overwrites the contents of the + * file a number of times. The file is then deleted. + * + * @param file File to be overwritten a number of times and, ultimately, + * deleted + * @param passes Number of times file should be overwritten + * @throws IOException if something makes shredding or deleting a problem + */ + public static void shredFile(final File file, final int passes) + throws IOException { + final Random generator = new Random(); + final long fileLength = file.length(); + final int byteArraySize = (int) Math.min(fileLength, 1048576); // 1MB + final byte[] b = new byte[byteArraySize]; + final long numOfRandomWrites = (fileLength / b.length) + 1; + final FileOutputStream fos = new FileOutputStream(file); + try { + // Over write file contents (passes) times + final FileChannel channel = fos.getChannel(); + for (int i = 0; i < passes; i++) { + generator.nextBytes(b); + for (int j = 0; j <= numOfRandomWrites; j++) { + fos.write(b); + } + fos.flush(); + channel.position(0); + } + // Write out "0" for each byte in the file + Arrays.fill(b, (byte) 0); + for (int j = 0; j < numOfRandomWrites; j++) { + fos.write(b); + } + fos.flush(); + fos.close(); + // Try to delete the file a few times + if (!FileUtils.deleteFile(file, null, 5)) { + throw new IOException("Failed to delete file after shredding"); + } + + } finally { + FileUtils.closeQuietly(fos); + } + } + + public static long copy(final InputStream in, final OutputStream out) throws IOException { + final byte[] buffer = new byte[65536]; + long copied = 0L; + int len; + while ((len = in.read(buffer)) > 0) { + out.write(buffer, 0, len); + copied += len; + } + + return copied; + } + + public static long copyBytes(final byte[] bytes, final File destination, final boolean lockOutputFile) throws FileNotFoundException, IOException { + FileOutputStream fos = null; + FileLock outLock = null; + long fileSize = 0L; + try { + fos = new FileOutputStream(destination); + final FileChannel out = fos.getChannel(); + if (lockOutputFile) { + outLock = out.tryLock(0, Long.MAX_VALUE, false); + if (null == outLock) { + throw new IOException("Unable to obtain exclusive file lock for: " + destination.getAbsolutePath()); + } + } + fos.write(bytes); + fos.flush(); + fileSize = bytes.length; + } finally { + FileUtils.releaseQuietly(outLock); + FileUtils.closeQuietly(fos); + } + return fileSize; + } + + /** + * Copies the given source file to the given destination file. The given + * destination will be overwritten if it already exists. + * + * @param source + * @param destination + * @param lockInputFile if true will lock input file during copy; if false + * will not + * @param lockOutputFile if true will lock output file during copy; if false + * will not + * @param move if true will perform what is effectively a move operation + * rather than a pure copy. This allows for potentially highly efficient + * movement of the file but if not possible this will revert to a copy then + * delete behavior. If false, then the file is copied and the source file is + * retained. If a true rename/move occurs then no lock is held during that + * time. + * @param logger if failures occur, they will be logged to this logger if + * possible. If this logger is null, an IOException will instead be thrown, + * indicating the problem. + * @return long number of bytes copied + * @throws FileNotFoundException if the source file could not be found + * @throws IOException + * @throws SecurityException if a security manager denies the needed file + * operations + */ + public static long copyFile(final File source, final File destination, final boolean lockInputFile, final boolean lockOutputFile, final boolean move, final Logger logger) throws FileNotFoundException, IOException { + + FileInputStream fis = null; + FileOutputStream fos = null; + FileLock inLock = null; + FileLock outLock = null; + long fileSize = 0L; + if (!source.canRead()) { + throw new IOException("Must at least have read permission"); + + } + if (move && source.renameTo(destination)) { + fileSize = destination.length(); + } else { + try { + fis = new FileInputStream(source); + fos = new FileOutputStream(destination); + final FileChannel in = fis.getChannel(); + final FileChannel out = fos.getChannel(); + if (lockInputFile) { + inLock = in.tryLock(0, Long.MAX_VALUE, true); + if (null == inLock) { + throw new IOException("Unable to obtain shared file lock for: " + source.getAbsolutePath()); + } + } + if (lockOutputFile) { + outLock = out.tryLock(0, Long.MAX_VALUE, false); + if (null == outLock) { + throw new IOException("Unable to obtain exclusive file lock for: " + destination.getAbsolutePath()); + } + } + long bytesWritten = 0; + do { + bytesWritten += out.transferFrom(in, bytesWritten, TRANSFER_CHUNK_SIZE_BYTES); + fileSize = in.size(); + } while (bytesWritten < fileSize); + out.force(false); + FileUtils.closeQuietly(fos); + FileUtils.closeQuietly(fis); + fos = null; + fis = null; + if (move && !FileUtils.deleteFile(source, null, 5)) { + if (logger == null) { + FileUtils.deleteFile(destination, null, 5); + throw new IOException("Could not remove file " + source.getAbsolutePath()); + } else { + logger.warn("Configured to delete source file when renaming/move not successful. However, unable to delete file at: " + source.getAbsolutePath()); + } + } + } finally { + FileUtils.releaseQuietly(inLock); + FileUtils.releaseQuietly(outLock); + FileUtils.closeQuietly(fos); + FileUtils.closeQuietly(fis); + } + } + return fileSize; + } + + /** + * Copies the given source file to the given destination file. The given + * destination will be overwritten if it already exists. + * + * @param source + * @param destination + * @param lockInputFile if true will lock input file during copy; if false + * will not + * @param lockOutputFile if true will lock output file during copy; if false + * will not + * @param logger + * @return long number of bytes copied + * @throws FileNotFoundException if the source file could not be found + * @throws IOException + * @throws SecurityException if a security manager denies the needed file + * operations + */ + public static long copyFile(final File source, final File destination, final boolean lockInputFile, final boolean lockOutputFile, final Logger logger) throws FileNotFoundException, IOException { + return FileUtils.copyFile(source, destination, lockInputFile, lockOutputFile, false, logger); + } + + public static long copyFile(final File source, final OutputStream stream, final boolean closeOutputStream, final boolean lockInputFile) throws FileNotFoundException, IOException { + FileInputStream fis = null; + FileLock inLock = null; + long fileSize = 0L; + try { + fis = new FileInputStream(source); + final FileChannel in = fis.getChannel(); + if (lockInputFile) { + inLock = in.tryLock(0, Long.MAX_VALUE, true); + if (inLock == null) { + throw new IOException("Unable to obtain exclusive file lock for: " + source.getAbsolutePath()); + } + + } + + byte[] buffer = new byte[1 << 18]; //256 KB + int bytesRead = -1; + while ((bytesRead = fis.read(buffer)) != -1) { + stream.write(buffer, 0, bytesRead); + } + in.force(false); + stream.flush(); + fileSize = in.size(); + } finally { + FileUtils.releaseQuietly(inLock); + FileUtils.closeQuietly(fis); + if (closeOutputStream) { + FileUtils.closeQuietly(stream); + } + } + return fileSize; + } + + public static long copyFile(final InputStream stream, final File destination, final boolean closeInputStream, final boolean lockOutputFile) throws FileNotFoundException, IOException { + final Path destPath = destination.toPath(); + final long size = Files.copy(stream, destPath); + if (closeInputStream) { + stream.close(); + } + return size; + } + + /** + * Renames the given file from the source path to the destination path. This + * handles multiple attempts. This should only be used to rename within a + * given directory. Renaming across directories might not work well. See the + * File.renameTo for more information. + * + * @param source the file to rename + * @param destination the file path to rename to + * @param maxAttempts the max number of attempts to attempt the rename + * @throws IOException if rename isn't successful + */ + public static void renameFile(final File source, final File destination, final int maxAttempts) throws IOException { + FileUtils.renameFile(source, destination, maxAttempts, false); + } + + /** + * Renames the given file from the source path to the destination path. This + * handles multiple attempts. This should only be used to rename within a + * given directory. Renaming across directories might not work well. See the + * File.renameTo for more information. + * + * @param source the file to rename + * @param destination the file path to rename to + * @param maxAttempts the max number of attempts to attempt the rename + * @param replace if true and a rename attempt fails will check if a file is + * already at the destination path. If so it will delete that file and + * attempt the rename according the remaining maxAttempts. If false, any + * conflicting files will be left as they were and the rename attempts will + * fail if conflicting. + * @throws IOException if rename isn't successful + */ + public static void renameFile(final File source, final File destination, final int maxAttempts, final boolean replace) throws IOException { + final int attempts = (replace || maxAttempts < 1) ? Math.max(2, maxAttempts) : maxAttempts; + boolean renamed = false; + for (int i = 0; i < attempts; i++) { + renamed = source.renameTo(destination); + if (!renamed) { + FileUtils.deleteFile(destination, null, 5); + } else { + break; //rename has succeeded + } + } + if (!renamed) { + throw new IOException("Attempted " + maxAttempts + " times but unable to rename from \'" + source.getPath() + "\' to \'" + destination.getPath() + "\'"); + + } + } + + public static void sleepQuietly(final long millis) { + try { + Thread.sleep(millis); + } catch (final InterruptedException ex) { + /* do nothing */ + } + } + + /** + * Syncs a primary copy of a file with the copy in the restore directory. If + * the restore directory does not have a file and the primary has a file, + * the the primary's file is copied to the restore directory. Else if the + * restore directory has a file, but the primary does not, then the + * restore's file is copied to the primary directory. Else if the primary + * file is different than the restore file, then an IllegalStateException is + * thrown. Otherwise, if neither file exists, then no syncing is performed. + * + * @param primaryFile the primary file + * @param restoreFile the restore file + * @param logger a logger + * @throws IOException if an I/O problem was encountered during syncing + * @throws IllegalStateException if the primary and restore copies exist but + * are different + */ + public static void syncWithRestore(final File primaryFile, final File restoreFile, final Logger logger) + throws IOException { + + if (primaryFile.exists() && !restoreFile.exists()) { + // copy primary file to restore + copyFile(primaryFile, restoreFile, false, false, logger); + } else if (restoreFile.exists() && !primaryFile.exists()) { + // copy restore file to primary + copyFile(restoreFile, primaryFile, false, false, logger); + } else if (primaryFile.exists() && restoreFile.exists() && !isSame(primaryFile, restoreFile)) { + throw new IllegalStateException(String.format("Primary file '%s' is different than restore file '%s'", + primaryFile.getAbsoluteFile(), restoreFile.getAbsolutePath())); + } + } + + /** + * Returns true if the given files are the same according to their MD5 hash. + * + * @param file1 a file + * @param file2 a file + * @return true if the files are the same; false otherwise + * @throws IOException if the MD5 hash could not be computed + */ + public static boolean isSame(final File file1, final File file2) throws IOException { + return Arrays.equals(computeMd5Digest(file1), computeMd5Digest(file2)); + } + + /** + * Returns the MD5 hash of the given file. + * + * @param file a file + * @return the MD5 hash + * @throws IOException if the MD5 hash could not be computed + */ + public static byte[] computeMd5Digest(final File file) throws IOException { + BufferedInputStream bis = null; + try { + bis = new BufferedInputStream(new FileInputStream(file)); + return DigestUtils.md5(bis); + } finally { + FileUtils.closeQuietly(bis); + } + } +} diff --git a/commons/nifi-logging-utils/pom.xml b/commons/nifi-logging-utils/pom.xml new file mode 100644 index 0000000000..ce5064b82f --- /dev/null +++ b/commons/nifi-logging-utils/pom.xml @@ -0,0 +1,35 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-logging-utils + 0.0.1-SNAPSHOT + NiFi Logging Utils + Utilities for logging + + + + org.slf4j + slf4j-api + + + diff --git a/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java b/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java new file mode 100644 index 0000000000..7c71d8569b --- /dev/null +++ b/commons/nifi-logging-utils/src/main/java/org/apache/nifi/logging/NiFiLog.java @@ -0,0 +1,367 @@ +/* + * 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.logging; + +import org.slf4j.Logger; +import org.slf4j.Marker; + +/** + * + * @author unattributed + */ +public class NiFiLog implements Logger { + + private final Logger logger; + + public NiFiLog(final Logger logger) { + this.logger = logger; + } + + public Logger getWrappedLog() { + return logger; + } + + @Override + public void warn(Marker marker, String string, Throwable thrwbl) { + if (logger.isDebugEnabled()) { + logger.warn(marker, string, thrwbl); + } else { + logger.warn(marker, string); + } + } + + @Override + public void warn(Marker marker, String string, Object[] os) { + logger.warn(marker, string, os); + } + + @Override + public void warn(Marker marker, String string, Object o, Object o1) { + logger.warn(marker, string, o, o1); + } + + @Override + public void warn(Marker marker, String string, Object o) { + logger.warn(marker, string, o); + } + + @Override + public void warn(Marker marker, String string) { + logger.warn(marker, string); + } + + @Override + public void warn(String string, Throwable thrwbl) { + if (logger.isDebugEnabled()) { + logger.warn(string, thrwbl); + } else { + logger.warn(string); + } + } + + @Override + public void warn(String string, Object o, Object o1) { + logger.warn(string, o, o1); + } + + @Override + public void warn(String string, Object[] os) { + logger.warn(string, os); + } + + @Override + public void warn(String string, Object o) { + logger.warn(string, o); + } + + @Override + public void warn(String string) { + logger.warn(string); + } + + @Override + public void trace(Marker marker, String string, Throwable thrwbl) { + logger.trace(marker, string, thrwbl); + } + + @Override + public void trace(Marker marker, String string, Object[] os) { + logger.trace(marker, string, os); + } + + @Override + public void trace(Marker marker, String string, Object o, Object o1) { + logger.trace(marker, string, o, o1); + } + + @Override + public void trace(Marker marker, String string, Object o) { + logger.trace(marker, string, o); + } + + @Override + public void trace(Marker marker, String string) { + logger.trace(marker, string); + } + + @Override + public void trace(String string, Throwable thrwbl) { + logger.trace(string, thrwbl); + } + + @Override + public void trace(String string, Object[] os) { + logger.trace(string, os); + } + + @Override + public void trace(String string, Object o, Object o1) { + logger.trace(string, o, o1); + } + + @Override + public void trace(String string, Object o) { + logger.trace(string, o); + } + + @Override + public void trace(String string) { + logger.trace(string); + } + + @Override + public boolean isWarnEnabled(Marker marker) { + return logger.isWarnEnabled(marker); + } + + @Override + public boolean isWarnEnabled() { + return logger.isWarnEnabled(); + } + + @Override + public boolean isTraceEnabled(Marker marker) { + return logger.isTraceEnabled(marker); + } + + @Override + public boolean isTraceEnabled() { + return logger.isTraceEnabled(); + } + + @Override + public boolean isInfoEnabled(Marker marker) { + return logger.isInfoEnabled(marker); + } + + @Override + public boolean isInfoEnabled() { + return logger.isInfoEnabled(); + } + + @Override + public boolean isErrorEnabled(Marker marker) { + return logger.isErrorEnabled(marker); + } + + @Override + public boolean isErrorEnabled() { + return logger.isErrorEnabled(); + } + + @Override + public boolean isDebugEnabled(Marker marker) { + return logger.isDebugEnabled(marker); + } + + @Override + public boolean isDebugEnabled() { + return logger.isDebugEnabled(); + } + + @Override + public void info(Marker marker, String string, Throwable thrwbl) { + if (logger.isDebugEnabled()) { + logger.info(marker, string, thrwbl); + } else { + logger.info(marker, string); + } + } + + @Override + public void info(Marker marker, String string, Object[] os) { + logger.info(marker, string, os); + } + + @Override + public void info(Marker marker, String string, Object o, Object o1) { + logger.info(marker, string, o, o1); + } + + @Override + public void info(Marker marker, String string, Object o) { + logger.info(marker, string, o); + } + + @Override + public void info(Marker marker, String string) { + logger.info(marker, string); + } + + @Override + public void info(String string, Throwable thrwbl) { + if (logger.isDebugEnabled()) { + logger.info(string, thrwbl); + } else { + logger.info(string); + } + } + + @Override + public void info(String string, Object[] os) { + logger.info(string, os); + } + + @Override + public void info(String string, Object o, Object o1) { + logger.info(string, o, o1); + } + + @Override + public void info(String string, Object o) { + logger.info(string, o); + } + + @Override + public void info(String string) { + logger.info(string); + } + + @Override + public String getName() { + return logger.getName(); + } + + @Override + public void error(Marker marker, String string, Throwable thrwbl) { + if (logger.isDebugEnabled()) { + logger.error(marker, string, thrwbl); + } else { + logger.error(marker, string); + } + } + + @Override + public void error(Marker marker, String string, Object[] os) { + logger.error(marker, string, os); + } + + @Override + public void error(Marker marker, String string, Object o, Object o1) { + logger.error(marker, string, o, o1); + } + + @Override + public void error(Marker marker, String string, Object o) { + logger.error(marker, string, o); + } + + @Override + public void error(Marker marker, String string) { + logger.error(marker, string); + } + + @Override + public void error(String string, Throwable thrwbl) { + if (logger.isDebugEnabled()) { + logger.error(string, thrwbl); + } else { + logger.error(string); + } + } + + @Override + public void error(String string, Object[] os) { + logger.error(string, os); + } + + @Override + public void error(String string, Object o, Object o1) { + logger.error(string, o, o1); + } + + @Override + public void error(String string, Object o) { + logger.error(string, o); + } + + @Override + public void error(String string) { + logger.error(string); + } + + @Override + public void debug(Marker marker, String string, Throwable thrwbl) { + logger.debug(marker, string, thrwbl); + } + + @Override + public void debug(Marker marker, String string, Object[] os) { + logger.debug(marker, string, os); + } + + @Override + public void debug(Marker marker, String string, Object o, Object o1) { + logger.debug(marker, string, o, o1); + } + + @Override + public void debug(Marker marker, String string, Object o) { + logger.debug(marker, string, o); + } + + @Override + public void debug(Marker marker, String string) { + logger.debug(marker, string); + } + + @Override + public void debug(String string, Throwable thrwbl) { + logger.debug(string, thrwbl); + } + + @Override + public void debug(String string, Object[] os) { + logger.debug(string, os); + } + + @Override + public void debug(String string, Object o, Object o1) { + logger.debug(string, o, o1); + } + + @Override + public void debug(String string, Object o) { + logger.debug(string, o); + } + + @Override + public void debug(String string) { + logger.debug(string); + } + +} diff --git a/commons/nifi-parent/pom.xml b/commons/nifi-parent/pom.xml new file mode 100644 index 0000000000..7684d53c52 --- /dev/null +++ b/commons/nifi-parent/pom.xml @@ -0,0 +1,217 @@ + + + + 4.0.0 + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + pom + NiFi Parent + + A helpful parent pom which can be used for all NiFi components. Helps establish the basic requirements/depdencies. + + + UTF-8 + 1.7.7 + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + 3.0.0 + + Max + Medium + true + + + + + check + + + + + + + + + org.apache.nifi + nar-maven-plugin + 0.0.1-SNAPSHOT + true + + + org.apache.maven.plugins + maven-compiler-plugin + + 1.7 + 1.7 + true + + 3.2 + + + maven-jar-plugin + 2.5 + + + maven-war-plugin + 2.5 + + + org.apache.maven.plugins + maven-dependency-plugin + 2.9 + + + org.apache.maven.plugins + maven-resources-plugin + 2.7 + + + org.apache.maven.plugins + maven-surefire-plugin + 2.18 + + + org.apache.maven.plugins + maven-assembly-plugin + 2.5.2 + + + org.apache.maven.plugins + maven-release-plugin + 2.5.1 + + + org.codehaus.mojo + jaxb2-maven-plugin + 1.6 + + + org.apache.maven.plugins + maven-source-plugin + 2.4 + + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + org.apache.maven.plugins + maven-site-plugin + 3.4 + + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.1 + + false + true + private + + + + + + + + ch.qos.logback + logback-classic + 1.1.2 + provided + + + org.slf4j + slf4j-api + + + + + org.slf4j + jcl-over-slf4j + ${org.slf4j.version} + provided + + + org.slf4j + jul-to-slf4j + ${org.slf4j.version} + provided + + + org.slf4j + slf4j-api + ${org.slf4j.version} + provided + + + + + + + junit + junit + 4.11 + test + + + org.mockito + mockito-core + 1.10.8 + test + + + org.slf4j + slf4j-simple + ${org.slf4j.version} + test + + + + + + nifi-releases + ${nifi.repo.url} + + + nifi-snapshots + ${nifi.snapshot.repo.url} + + + + + + org.codehaus.mojo + findbugs-maven-plugin + 3.0.0 + + Max + Medium + true + + + + + diff --git a/commons/nifi-properties/.gitignore b/commons/nifi-properties/.gitignore new file mode 100755 index 0000000000..073c9faaea --- /dev/null +++ b/commons/nifi-properties/.gitignore @@ -0,0 +1,3 @@ +/target +/target +/target diff --git a/commons/nifi-properties/pom.xml b/commons/nifi-properties/pom.xml new file mode 100644 index 0000000000..70f90aa0b7 --- /dev/null +++ b/commons/nifi-properties/pom.xml @@ -0,0 +1,29 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + nifi-properties + 0.0.1-SNAPSHOT + NiFi Properties + + + + diff --git a/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java new file mode 100644 index 0000000000..15208146a6 --- /dev/null +++ b/commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -0,0 +1,882 @@ +/* + * 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.util; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedInputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.InputStream; +import java.net.InetSocketAddress; +import java.nio.file.InvalidPathException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class NiFiProperties extends Properties { + + private static final long serialVersionUID = 2119177359005492702L; + + private static final Logger LOG = LoggerFactory.getLogger(NiFiProperties.class); + private static NiFiProperties instance = null; + + // core properties + public static final String PROPERTIES_FILE_PATH = "nifi.properties.file.path"; + public static final String FLOW_CONFIGURATION_FILE = "nifi.flow.configuration.file"; + public static final String FLOW_CONFIGURATION_ARCHIVE_FILE = "nifi.flow.configuration.archive.file"; + public static final String TASK_CONFIGURATION_FILE = "nifi.reporting.task.configuration.file"; + public static final String SERVICE_CONFIGURATION_FILE = "nifi.controller.service.configuration.file"; + public static final String AUTHORITY_PROVIDER_CONFIGURATION_FILE = "nifi.authority.provider.configuration.file"; + public static final String REPOSITORY_DATABASE_DIRECTORY = "nifi.database.directory"; + public static final String RESTORE_DIRECTORY = "nifi.restore.directory"; + public static final String VERSION = "nifi.version"; + public static final String WRITE_DELAY_INTERVAL = "nifi.flowservice.writedelay.interval"; + public static final String AUTO_RESUME_STATE = "nifi.flowcontroller.autoResumeState"; + public static final String FLOW_CONTROLLER_GRACEFUL_SHUTDOWN_PERIOD = "nifi.flowcontroller.graceful.shutdown.period"; + public static final String NAR_LIBRARY_DIRECTORY = "nifi.nar.library.directory"; + public static final String NAR_WORKING_DIRECTORY = "nifi.nar.working.directory"; + public static final String COMPONENT_DOCS_DIRECTORY = "nifi.documentation.working.directory"; + public static final String SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key"; + public static final String SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm"; + public static final String SENSITIVE_PROPS_PROVIDER = "nifi.sensitive.props.provider"; + public static final String H2_URL_APPEND = "nifi.h2.url.append"; + public static final String REMOTE_INPUT_PORT = "nifi.remote.input.socket.port"; + public static final String SITE_TO_SITE_SECURE = "nifi.remote.input.secure"; + public static final String TEMPLATE_DIRECTORY = "nifi.templates.directory"; + public static final String ADMINISTRATIVE_YIELD_DURATION = "nifi.administrative.yield.duration"; + public static final String PERSISTENT_STATE_DIRECTORY = "nifi.persistent.state.directory"; + + // content repository properties + public static final String REPOSITORY_CONTENT_PREFIX = "nifi.content.repository.directory."; + public static final String CONTENT_REPOSITORY_IMPLEMENTATION = "nifi.content.repository.implementation"; + public static final String MAX_APPENDABLE_CLAIM_SIZE = "nifi.content.claim.max.appendable.size"; + public static final String MAX_FLOWFILES_PER_CLAIM = "nifi.content.claim.max.flow.files"; + public static final String CONTENT_ARCHIVE_MAX_RETENTION_PERIOD = "nifi.content.repository.archive.max.retention.period"; + public static final String CONTENT_ARCHIVE_MAX_USAGE_PERCENTAGE = "nifi.content.repository.archive.max.usage.percentage"; + public static final String CONTENT_ARCHIVE_BACK_PRESSURE_PERCENTAGE = "nifi.content.repository.archive.backpressure.percentage"; + public static final String CONTENT_ARCHIVE_ENABLED = "nifi.content.repository.archive.enabled"; + public static final String CONTENT_ARCHIVE_CLEANUP_FREQUENCY = "nifi.content.repository.archive.cleanup.frequency"; + public static final String CONTENT_VIEWER_URL = "nifi.content.viewer.url"; + + // flowfile repository properties + public static final String FLOWFILE_REPOSITORY_IMPLEMENTATION = "nifi.flowfile.repository.implementation"; + public static final String FLOWFILE_REPOSITORY_ALWAYS_SYNC = "nifi.flowfile.repository.always.sync"; + public static final String FLOWFILE_REPOSITORY_DIRECTORY = "nifi.flowfile.repository.directory"; + public static final String FLOWFILE_REPOSITORY_PARTITIONS = "nifi.flowfile.repository.partitions"; + public static final String FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL = "nifi.flowfile.repository.checkpoint.interval"; + public static final String FLOWFILE_SWAP_MANAGER_IMPLEMENTATION = "nifi.swap.manager.implementation"; + public static final String QUEUE_SWAP_THRESHOLD = "nifi.queue.swap.threshold"; + public static final String SWAP_STORAGE_LOCATION = "nifi.swap.storage.directory"; + public static final String SWAP_IN_THREADS = "nifi.swap.in.threads"; + public static final String SWAP_IN_PERIOD = "nifi.swap.in.period"; + public static final String SWAP_OUT_THREADS = "nifi.swap.out.threads"; + public static final String SWAP_OUT_PERIOD = "nifi.swap.out.period"; + + // provenance properties + public static final String PROVENANCE_REPO_IMPLEMENTATION_CLASS = "nifi.provenance.repository.implementation"; + public static final String PROVENANCE_REPO_DIRECTORY_PREFIX = "nifi.provenance.repository.directory."; + public static final String PROVENANCE_MAX_STORAGE_TIME = "nifi.provenance.repository.max.storage.time"; + public static final String PROVENANCE_MAX_STORAGE_SIZE = "nifi.provenance.repository.max.storage.size"; + public static final String PROVENANCE_ROLLOVER_TIME = "nifi.provenance.repository.rollover.time"; + public static final String PROVENANCE_ROLLOVER_SIZE = "nifi.provenance.repository.rollover.size"; + public static final String PROVENANCE_QUERY_THREAD_POOL_SIZE = "nifi.provenance.repository.query.threads"; + public static final String PROVENANCE_COMPRESS_ON_ROLLOVER = "nifi.provenance.repository.compress.on.rollover"; + public static final String PROVENANCE_INDEXED_FIELDS = "nifi.provenance.repository.indexed.fields"; + public static final String PROVENANCE_INDEXED_ATTRIBUTES = "nifi.provenance.repository.indexed.attributes"; + public static final String PROVENANCE_INDEX_SHARD_SIZE = "nifi.provenance.repository.index.shard.size"; + public static final String PROVENANCE_JOURNAL_COUNT = "nifi.provenance.repository.journal.count"; + + // component status repository properties + public static final String COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION = "nifi.components.status.repository.implementation"; + public static final String COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "nifi.components.status.snapshot.frequency"; + + // encryptor properties + public static final String NF_SENSITIVE_PROPS_KEY = "nifi.sensitive.props.key"; + public static final String NF_SENSITIVE_PROPS_ALGORITHM = "nifi.sensitive.props.algorithm"; + public static final String NF_SENSITIVE_PROPS_PROVIDER = "nifi.sensitive.props.provider"; + + // security properties + public static final String SECURITY_KEYSTORE = "nifi.security.keystore"; + public static final String SECURITY_KEYSTORE_TYPE = "nifi.security.keystoreType"; + public static final String SECURITY_KEYSTORE_PASSWD = "nifi.security.keystorePasswd"; + public static final String SECURITY_KEY_PASSWD = "nifi.security.keyPasswd"; + public static final String SECURITY_TRUSTSTORE = "nifi.security.truststore"; + public static final String SECURITY_TRUSTSTORE_TYPE = "nifi.security.truststoreType"; + public static final String SECURITY_TRUSTSTORE_PASSWD = "nifi.security.truststorePasswd"; + public static final String SECURITY_NEED_CLIENT_AUTH = "nifi.security.needClientAuth"; + public static final String SECURITY_USER_AUTHORITY_PROVIDER = "nifi.security.user.authority.provider"; + public static final String SECURITY_CLUSTER_AUTHORITY_PROVIDER_PORT = "nifi.security.cluster.authority.provider.port"; + public static final String SECURITY_CLUSTER_AUTHORITY_PROVIDER_THREADS = "nifi.security.cluster.authority.provider.threads"; + public static final String SECURITY_USER_CREDENTIAL_CACHE_DURATION = "nifi.security.user.credential.cache.duration"; + public static final String SECURITY_SUPPORT_NEW_ACCOUNT_REQUESTS = "nifi.security.support.new.account.requests"; + public static final String SECURITY_DEFAULT_USER_ROLES = "nifi.security.default.user.roles"; + public static final String SECURITY_OCSP_RESPONDER_URL = "nifi.security.ocsp.responder.url"; + public static final String SECURITY_OCSP_RESPONDER_CERTIFICATE = "nifi.security.ocsp.responder.certificate"; + + // web properties + public static final String WEB_WAR_DIR = "nifi.web.war.directory"; + public static final String WEB_HTTP_PORT = "nifi.web.http.port"; + public static final String WEB_HTTP_HOST = "nifi.web.http.host"; + public static final String WEB_HTTPS_PORT = "nifi.web.https.port"; + public static final String WEB_HTTPS_HOST = "nifi.web.https.host"; + public static final String WEB_WORKING_DIR = "nifi.web.jetty.working.directory"; + + // ui properties + public static final String UI_BANNER_TEXT = "nifi.ui.banner.text"; + public static final String UI_AUTO_REFRESH_INTERVAL = "nifi.ui.autorefresh.interval"; + + // cluster common properties + public static final String CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "nifi.cluster.protocol.heartbeat.interval"; + public static final String CLUSTER_PROTOCOL_IS_SECURE = "nifi.cluster.protocol.is.secure"; + public static final String CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "nifi.cluster.protocol.socket.timeout"; + public static final String CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "nifi.cluster.protocol.connection.handshake.timeout"; + public static final String CLUSTER_PROTOCOL_USE_MULTICAST = "nifi.cluster.protocol.use.multicast"; + public static final String CLUSTER_PROTOCOL_MULTICAST_ADDRESS = "nifi.cluster.protocol.multicast.address"; + public static final String CLUSTER_PROTOCOL_MULTICAST_PORT = "nifi.cluster.protocol.multicast.port"; + public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "nifi.cluster.protocol.multicast.service.broadcast.delay"; + public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = "nifi.cluster.protocol.multicast.service.locator.attempts"; + public static final String CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "nifi.cluster.protocol.multicast.service.locator.attempts.delay"; + + // cluster node properties + public static final String CLUSTER_IS_NODE = "nifi.cluster.is.node"; + public static final String CLUSTER_NODE_ADDRESS = "nifi.cluster.node.address"; + public static final String CLUSTER_NODE_PROTOCOL_PORT = "nifi.cluster.node.protocol.port"; + public static final String CLUSTER_NODE_PROTOCOL_THREADS = "nifi.cluster.node.protocol.threads"; + public static final String CLUSTER_NODE_UNICAST_MANAGER_ADDRESS = "nifi.cluster.node.unicast.manager.address"; + public static final String CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT = "nifi.cluster.node.unicast.manager.protocol.port"; + + // cluster manager properties + public static final String CLUSTER_IS_MANAGER = "nifi.cluster.is.manager"; + public static final String CLUSTER_MANAGER_ADDRESS = "nifi.cluster.manager.address"; + public static final String CLUSTER_MANAGER_PROTOCOL_PORT = "nifi.cluster.manager.protocol.port"; + public static final String CLUSTER_MANAGER_NODE_FIREWALL_FILE = "nifi.cluster.manager.node.firewall.file"; + public static final String CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = "nifi.cluster.manager.node.event.history.size"; + public static final String CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "nifi.cluster.manager.node.api.connection.timeout"; + public static final String CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "nifi.cluster.manager.node.api.read.timeout"; + public static final String CLUSTER_MANAGER_NODE_API_REQUEST_THREADS = "nifi.cluster.manager.node.api.request.threads"; + public static final String CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "nifi.cluster.manager.flow.retrieval.delay"; + public static final String CLUSTER_MANAGER_PROTOCOL_THREADS = "nifi.cluster.manager.protocol.threads"; + public static final String CLUSTER_MANAGER_SAFEMODE_DURATION = "nifi.cluster.manager.safemode.duration"; + + // defaults + public static final String DEFAULT_TITLE = "NiFi"; + public static final Boolean DEFAULT_AUTO_RESUME_STATE = true; + public static final String DEFAULT_AUTHORITY_PROVIDER_CONFIGURATION_FILE = "conf/authority-providers.xml"; + public static final String DEFAULT_USER_CREDENTIAL_CACHE_DURATION = "24 hours"; + public static final Integer DEFAULT_REMOTE_INPUT_PORT = null; + public static final Path DEFAULT_TEMPLATE_DIRECTORY = Paths.get("conf", "templates"); + public static final String DEFAULT_WEB_WORKING_DIR = "./work/jetty"; + public static final String DEFAULT_NAR_WORKING_DIR = "./work/nar"; + public static final String DEFAULT_COMPONENT_DOCS_DIRECTORY = "./work/docs/components"; + public static final String DEFAULT_NAR_LIBRARY_DIR = "./lib"; + public static final String DEFAULT_FLOWFILE_REPO_PARTITIONS = "256"; + public static final String DEFAULT_FLOWFILE_CHECKPOINT_INTERVAL = "2 min"; + public static final int DEFAULT_MAX_FLOWFILES_PER_CLAIM = 100; + public static final int DEFAULT_QUEUE_SWAP_THRESHOLD = 20000; + public static final String DEFAULT_SWAP_STORAGE_LOCATION = "./flowfile_repository/swap"; + public static final String DEFAULT_SWAP_IN_PERIOD = "1 sec"; + public static final String DEFAULT_SWAP_OUT_PERIOD = "5 sec"; + public static final int DEFAULT_SWAP_IN_THREADS = 4; + public static final int DEFAULT_SWAP_OUT_THREADS = 4; + public static final String DEFAULT_ADMINISTRATIVE_YIELD_DURATION = "30 sec"; + public static final String DEFAULT_PERSISTENT_STATE_DIRECTORY = "./conf/state"; + public static final String DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY = "5 mins"; + + // cluster common defaults + public static final String DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL = "5 sec"; + public static final String DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY = "500 ms"; + public static final int DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS = 3; + public static final String DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY = "1 sec"; + public static final String DEFAULT_CLUSTER_PROTOCOL_SOCKET_TIMEOUT = "30 sec"; + public static final String DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT = "45 sec"; + + // cluster node defaults + public static final int DEFAULT_CLUSTER_NODE_PROTOCOL_THREADS = 2; + + // cluster manager defaults + public static final int DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE = 10; + public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT = "30 sec"; + public static final String DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT = "30 sec"; + public static final int DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS = 10; + public static final String DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY = "5 sec"; + public static final int DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS = 10; + public static final String DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION = "0 sec"; + + private NiFiProperties() { + super(); + } + + /** + * This is the method through which the NiFiProperties object should be + * obtained. + * + * @return the NiFiProperties object to use + * @throws RuntimeException if unable to load properties file + */ + public static synchronized NiFiProperties getInstance() { + if (null == instance) { + final NiFiProperties suspectInstance = new NiFiProperties(); + final String nfPropertiesFilePath = System.getProperty(NiFiProperties.PROPERTIES_FILE_PATH); + if (null == nfPropertiesFilePath || nfPropertiesFilePath.trim().length() == 0) { + throw new RuntimeException("Requires a system property called \'" + NiFiProperties.PROPERTIES_FILE_PATH + "\' and this is not set or has no value"); + } + final File propertiesFile = new File(nfPropertiesFilePath); + if (!propertiesFile.exists()) { + throw new RuntimeException("Properties file doesn't exist \'" + propertiesFile.getAbsolutePath() + "\'"); + } + if (!propertiesFile.canRead()) { + throw new RuntimeException("Properties file exists but cannot be read \'" + propertiesFile.getAbsolutePath() + "\'"); + } + InputStream inStream = null; + try { + inStream = new BufferedInputStream(new FileInputStream(propertiesFile)); + suspectInstance.load(inStream); + } catch (final Exception ex) { + LOG.error("Cannot load properties file due to " + ex.getLocalizedMessage()); + throw new RuntimeException("Cannot load properties file due to " + ex.getLocalizedMessage(), ex); + } finally { + if (null != inStream) { + try { + inStream.close(); + } catch (final Exception ex) { + /** + * do nothing * + */ + } + } + } + instance = suspectInstance; + } + return instance; + } + + // getters for core properties // + public File getFlowConfigurationFile() { + try { + return new File(getProperty(FLOW_CONFIGURATION_FILE)); + } catch (Exception ex) { + return null; + } + } + + public File getFlowConfigurationFileDir() { + try { + return getFlowConfigurationFile().getParentFile(); + } catch (Exception ex) { + return null; + } + } + + private Integer getPropertyAsPort(final String propertyName, final Integer defaultValue) { + final String port = getProperty(propertyName); + if (StringUtils.isEmpty(port)) { + return defaultValue; + } + try { + final int val = Integer.parseInt(port); + if (val <= 0 || val > 65535) { + throw new RuntimeException("Valid port range is 0 - 65535 but got " + val); + } + return val; + } catch (final NumberFormatException e) { + return defaultValue; + } + } + + public int getQueueSwapThreshold() { + final String thresholdValue = getProperty(QUEUE_SWAP_THRESHOLD); + if (thresholdValue == null) { + return DEFAULT_QUEUE_SWAP_THRESHOLD; + } + + try { + return Integer.parseInt(thresholdValue); + } catch (final NumberFormatException e) { + return DEFAULT_QUEUE_SWAP_THRESHOLD; + } + } + + public File getSwapStorageLocation() { + final String location = getProperty(SWAP_STORAGE_LOCATION); + if (location == null) { + return new File(DEFAULT_SWAP_STORAGE_LOCATION); + } else { + return new File(location); + } + } + + public Integer getIntegerProperty(final String propertyName, final Integer defaultValue) { + final String value = getProperty(propertyName); + if (value == null) { + return defaultValue; + } + + try { + return Integer.parseInt(getProperty(propertyName)); + } catch (final Exception e) { + return defaultValue; + } + } + + public int getSwapInThreads() { + return getIntegerProperty(SWAP_IN_THREADS, DEFAULT_SWAP_IN_THREADS); + } + + public int getSwapOutThreads() { + final String value = getProperty(SWAP_OUT_THREADS); + if (value == null) { + return DEFAULT_SWAP_OUT_THREADS; + } + + try { + return Integer.parseInt(getProperty(SWAP_OUT_THREADS)); + } catch (final Exception e) { + return DEFAULT_SWAP_OUT_THREADS; + } + } + + public String getSwapInPeriod() { + return getProperty(SWAP_IN_PERIOD, DEFAULT_SWAP_IN_PERIOD); + } + + public String getSwapOutPeriod() { + return getProperty(SWAP_OUT_PERIOD, DEFAULT_SWAP_OUT_PERIOD); + } + + public String getAdministrativeYieldDuration() { + return getProperty(ADMINISTRATIVE_YIELD_DURATION, DEFAULT_ADMINISTRATIVE_YIELD_DURATION); + } + + /** + * The socket port to listen on for a Remote Input Port. + * + * @return + */ + public Integer getRemoteInputPort() { + return getPropertyAsPort(REMOTE_INPUT_PORT, DEFAULT_REMOTE_INPUT_PORT); + } + + public Boolean isSiteToSiteSecure() { + final String secureVal = getProperty(SITE_TO_SITE_SECURE); + if (secureVal == null) { + return null; + } + + if ("true".equalsIgnoreCase(secureVal)) { + return true; + } + if ("false".equalsIgnoreCase(secureVal)) { + return false; + } + + throw new IllegalStateException("Property value for " + SITE_TO_SITE_SECURE + " is " + secureVal + "; expected 'true' or 'false'"); + } + + /** + * Returns the directory to which Templates are to be persisted + * + * @return + */ + public Path getTemplateDirectory() { + final String strVal = getProperty(TEMPLATE_DIRECTORY); + return (strVal == null) ? DEFAULT_TEMPLATE_DIRECTORY : Paths.get(strVal); + } + + /** + * Get the flow service write delay. + * + * @return The write delay + */ + public String getFlowServiceWriteDelay() { + return getProperty(WRITE_DELAY_INTERVAL); + } + + /** + * Returns whether the processors should be started automatically when the + * application loads. + * + * @return Whether to auto start the processors or not + */ + public boolean getAutoResumeState() { + final String rawAutoResumeState = getProperty(AUTO_RESUME_STATE, DEFAULT_AUTO_RESUME_STATE.toString()); + return Boolean.parseBoolean(rawAutoResumeState); + } + + /** + * Returns the number of partitions that should be used for the FlowFile + * Repository + * + * @return + */ + public int getFlowFileRepositoryPartitions() { + final String rawProperty = getProperty(FLOWFILE_REPOSITORY_PARTITIONS, DEFAULT_FLOWFILE_REPO_PARTITIONS); + return Integer.parseInt(rawProperty); + } + + /** + * Returns the number of milliseconds between FlowFileRepository + * checkpointing + * + * @return + */ + public String getFlowFileRepositoryCheckpointInterval() { + return getProperty(FLOWFILE_REPOSITORY_CHECKPOINT_INTERVAL, DEFAULT_FLOWFILE_CHECKPOINT_INTERVAL); + } + + /** + * @return the restore directory or null if not configured + */ + public File getRestoreDirectory() { + final String value = getProperty(RESTORE_DIRECTORY); + if (StringUtils.isBlank(value)) { + return null; + } else { + return new File(value); + } + } + + /** + * @return the user authorities file + */ + public File getAuthorityProviderConfiguraitonFile() { + final String value = getProperty(AUTHORITY_PROVIDER_CONFIGURATION_FILE); + if (StringUtils.isBlank(value)) { + return new File(DEFAULT_AUTHORITY_PROVIDER_CONFIGURATION_FILE); + } else { + return new File(value); + } + } + + /** + * Will default to true unless the value is explicitly set to false. + * + * @return Whether client auth is required + */ + public boolean getNeedClientAuth() { + boolean needClientAuth = true; + String rawNeedClientAuth = getProperty(SECURITY_NEED_CLIENT_AUTH); + if ("false".equalsIgnoreCase(rawNeedClientAuth)) { + needClientAuth = false; + } + return needClientAuth; + } + + public String getUserCredentialCacheDuration() { + return getProperty(SECURITY_USER_CREDENTIAL_CACHE_DURATION, DEFAULT_USER_CREDENTIAL_CACHE_DURATION); + } + + public boolean getSupportNewAccountRequests() { + boolean shouldSupport = true; + String rawShouldSupport = getProperty(SECURITY_SUPPORT_NEW_ACCOUNT_REQUESTS); + if ("false".equalsIgnoreCase(rawShouldSupport)) { + shouldSupport = false; + } + return shouldSupport; + } + + // getters for web properties // + public Integer getPort() { + Integer port = null; + try { + port = Integer.parseInt(getProperty(WEB_HTTP_PORT)); + } catch (NumberFormatException nfe) { + } + return port; + } + + public Integer getSslPort() { + Integer sslPort = null; + try { + sslPort = Integer.parseInt(getProperty(WEB_HTTPS_PORT)); + } catch (NumberFormatException nfe) { + } + return sslPort; + } + + public File getWebWorkingDirectory() { + return new File(getProperty(WEB_WORKING_DIR, DEFAULT_WEB_WORKING_DIR)); + } + + public File getComponentDocumentationWorkingDirectory() { + return new File(getProperty(COMPONENT_DOCS_DIRECTORY, DEFAULT_COMPONENT_DOCS_DIRECTORY)); + } + + public File getNarWorkingDirectory() { + return new File(getProperty(NAR_WORKING_DIRECTORY, DEFAULT_NAR_WORKING_DIR)); + } + + public File getFrameworkWorkingDirectory() { + return new File(getNarWorkingDirectory(), "framework"); + } + + public File getExtensionsWorkingDirectory() { + return new File(getNarWorkingDirectory(), "extensions"); + } + + public File getNarLibraryDirectory() { + return new File(getProperty(NAR_LIBRARY_DIRECTORY, DEFAULT_NAR_LIBRARY_DIR)); + } + + // getters for ui properties // + /** + * Get the title for the UI. + * + * @return The UI title + */ + public String getUiTitle() { + return this.getProperty(VERSION, DEFAULT_TITLE); + } + + /** + * Get the banner text. + * + * @return The banner text + */ + public String getBannerText() { + return this.getProperty(UI_BANNER_TEXT, StringUtils.EMPTY); + } + + /** + * Returns the auto refresh interval in seconds. + * + * @return + */ + public String getAutoRefreshInterval() { + return getProperty(UI_AUTO_REFRESH_INTERVAL); + } + + // getters for cluster protocol properties // + public String getClusterProtocolHeartbeatInterval() { + return getProperty(CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL, DEFAULT_CLUSTER_PROTOCOL_HEARTBEAT_INTERVAL); + } + + public String getNodeHeartbeatInterval() { + return getClusterProtocolHeartbeatInterval(); + } + + public String getClusterProtocolSocketTimeout() { + return getProperty(CLUSTER_PROTOCOL_SOCKET_TIMEOUT, DEFAULT_CLUSTER_PROTOCOL_SOCKET_TIMEOUT); + } + + public String getClusterProtocolConnectionHandshakeTimeout() { + return getProperty(CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT, DEFAULT_CLUSTER_PROTOCOL_CONNECTION_HANDSHAKE_TIMEOUT); + } + + public boolean getClusterProtocolUseMulticast() { + return Boolean.parseBoolean(getProperty(CLUSTER_PROTOCOL_USE_MULTICAST)); + } + + public InetSocketAddress getClusterProtocolMulticastAddress() { + try { + String multicastAddress = getProperty(CLUSTER_PROTOCOL_MULTICAST_ADDRESS); + int multicastPort = Integer.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_PORT)); + return new InetSocketAddress(multicastAddress, multicastPort); + } catch (Exception ex) { + throw new RuntimeException("Invalid multicast address/port due to: " + ex, ex); + } + } + + public String getClusterProtocolMulticastServiceBroadcastDelay() { + return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_BROADCAST_DELAY); + } + + public File getPersistentStateDirectory() { + final String dirName = getProperty(PERSISTENT_STATE_DIRECTORY, DEFAULT_PERSISTENT_STATE_DIRECTORY); + final File file = new File(dirName); + if (!file.exists()) { + file.mkdirs(); + } + return file; + } + + public int getClusterProtocolMulticastServiceLocatorAttempts() { + try { + return Integer.parseInt(getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS)); + } catch (NumberFormatException nfe) { + return DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS; + } + } + + public String getClusterProtocolMulticastServiceLocatorAttemptsDelay() { + return getProperty(CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY, DEFAULT_CLUSTER_PROTOCOL_MULTICAST_SERVICE_LOCATOR_ATTEMPTS_DELAY); + } + + // getters for cluster node properties // + public boolean isNode() { + return Boolean.parseBoolean(getProperty(CLUSTER_IS_NODE)); + } + + public InetSocketAddress getClusterNodeProtocolAddress() { + try { + String socketAddress = getProperty(CLUSTER_NODE_ADDRESS); + if (StringUtils.isBlank(socketAddress)) { + socketAddress = "localhost"; + } + int socketPort = getClusterNodeProtocolPort(); + return InetSocketAddress.createUnresolved(socketAddress, socketPort); + } catch (Exception ex) { + throw new RuntimeException("Invalid node protocol address/port due to: " + ex, ex); + } + } + + public Integer getClusterNodeProtocolPort() { + try { + return Integer.parseInt(getProperty(CLUSTER_NODE_PROTOCOL_PORT)); + } catch (NumberFormatException nfe) { + return null; + } + } + + public int getClusterNodeProtocolThreads() { + try { + return Integer.parseInt(getProperty(CLUSTER_NODE_PROTOCOL_THREADS)); + } catch (NumberFormatException nfe) { + return DEFAULT_CLUSTER_NODE_PROTOCOL_THREADS; + } + } + + public InetSocketAddress getClusterNodeUnicastManagerProtocolAddress() { + try { + String socketAddress = getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS); + if (StringUtils.isBlank(socketAddress)) { + socketAddress = "localhost"; + } + int socketPort = Integer.parseInt(getProperty(CLUSTER_NODE_UNICAST_MANAGER_PROTOCOL_PORT)); + return InetSocketAddress.createUnresolved(socketAddress, socketPort); + } catch (Exception ex) { + throw new RuntimeException("Invalid unicast manager address/port due to: " + ex, ex); + } + } + + // getters for cluster manager properties // + public boolean isClusterManager() { + return Boolean.parseBoolean(getProperty(CLUSTER_IS_MANAGER)); + } + + public InetSocketAddress getClusterManagerProtocolAddress() { + try { + String socketAddress = getProperty(CLUSTER_MANAGER_ADDRESS); + if (StringUtils.isBlank(socketAddress)) { + socketAddress = "localhost"; + } + int socketPort = getClusterManagerProtocolPort(); + return InetSocketAddress.createUnresolved(socketAddress, socketPort); + } catch (Exception ex) { + throw new RuntimeException("Invalid manager protocol address/port due to: " + ex, ex); + } + } + + public Integer getClusterManagerProtocolPort() { + try { + return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_PORT)); + } catch (NumberFormatException nfe) { + return null; + } + } + + public File getClusterManagerNodeFirewallFile() { + final String firewallFile = getProperty(CLUSTER_MANAGER_NODE_FIREWALL_FILE); + if (StringUtils.isBlank(firewallFile)) { + return null; + } else { + return new File(firewallFile); + } + } + + public int getClusterManagerNodeEventHistorySize() { + try { + return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE)); + } catch (NumberFormatException nfe) { + return DEFAULT_CLUSTER_MANAGER_NODE_EVENT_HISTORY_SIZE; + } + } + + public String getClusterManagerNodeApiConnectionTimeout() { + return getProperty(CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT, DEFAULT_CLUSTER_MANAGER_NODE_API_CONNECTION_TIMEOUT); + } + + public String getClusterManagerNodeApiReadTimeout() { + return getProperty(CLUSTER_MANAGER_NODE_API_READ_TIMEOUT, DEFAULT_CLUSTER_MANAGER_NODE_API_READ_TIMEOUT); + } + + public int getClusterManagerNodeApiRequestThreads() { + try { + return Integer.parseInt(getProperty(CLUSTER_MANAGER_NODE_API_REQUEST_THREADS)); + } catch (NumberFormatException nfe) { + return DEFAULT_CLUSTER_MANAGER_NODE_API_NUM_REQUEST_THREADS; + } + } + + public String getClusterManagerFlowRetrievalDelay() { + return getProperty(CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY, DEFAULT_CLUSTER_MANAGER_FLOW_RETRIEVAL_DELAY); + } + + public int getClusterManagerProtocolThreads() { + try { + return Integer.parseInt(getProperty(CLUSTER_MANAGER_PROTOCOL_THREADS)); + } catch (NumberFormatException nfe) { + return DEFAULT_CLUSTER_MANAGER_PROTOCOL_THREADS; + } + } + + public String getClusterManagerSafeModeDuration() { + return getProperty(CLUSTER_MANAGER_SAFEMODE_DURATION, DEFAULT_CLUSTER_MANAGER_SAFEMODE_DURATION); + } + + public String getClusterProtocolManagerToNodeApiScheme() { + final String isSecureProperty = getProperty(CLUSTER_PROTOCOL_IS_SECURE); + if (Boolean.valueOf(isSecureProperty)) { + return "https"; + } else { + return "http"; + } + } + + public InetSocketAddress getNodeApiAddress() { + + final String rawScheme = getClusterProtocolManagerToNodeApiScheme(); + final String scheme = (rawScheme == null) ? "http" : rawScheme; + + final String host; + final int port; + if ("http".equalsIgnoreCase(scheme)) { + // get host + if (StringUtils.isBlank(getProperty(WEB_HTTP_HOST))) { + host = "localhost"; + } else { + host = getProperty(WEB_HTTP_HOST); + } + // get port + port = getPort(); + } else { + // get host + if (StringUtils.isBlank(getProperty(WEB_HTTPS_HOST))) { + host = "localhost"; + } else { + host = getProperty(WEB_HTTPS_HOST); + } + // get port + port = getSslPort(); + } + + return InetSocketAddress.createUnresolved(host, port); + + } + + /** + * Returns the database repository path. It simply returns the value + * configured. No directories will be created as a result of this operation. + * + * @return database repository path + * @throws InvalidPathException If the configured path is invalid + */ + public Path getDatabaseRepositoryPath() { + return Paths.get(getProperty(REPOSITORY_DATABASE_DIRECTORY)); + } + + /** + * Returns the flow file repository path. It simply returns the value + * configured. No directories will be created as a result of this operation. + * + * @return database repository path + * @throws InvalidPathException If the configured path is invalid + */ + public Path getFlowFileRepositoryPath() { + return Paths.get(getProperty(FLOWFILE_REPOSITORY_DIRECTORY)); + } + + /** + * Returns the content repository paths. This method returns a mapping of + * file repository name to file repository paths. It simply returns the + * values configured. No directories will be created as a result of this + * operation. + * + * @return file repositories paths + * @throws InvalidPathException If any of the configured paths are invalid + */ + public Map getContentRepositoryPaths() { + final Map contentRepositoryPaths = new HashMap<>(); + + // go through each property + for (String propertyName : stringPropertyNames()) { + // determine if the property is a file repository path + if (StringUtils.startsWith(propertyName, REPOSITORY_CONTENT_PREFIX)) { + // get the repository key + final String key = StringUtils.substringAfter(propertyName, REPOSITORY_CONTENT_PREFIX); + + // attempt to resolve the path specified + contentRepositoryPaths.put(key, Paths.get(getProperty(propertyName))); + } + } + return contentRepositoryPaths; + } + + /** + * Returns the provenance repository paths. This method returns a mapping of + * file repository name to file repository paths. It simply returns the + * values configured. No directories will be created as a result of this + * operation. + * + * @return + */ + public Map getProvenanceRepositoryPaths() { + final Map provenanceRepositoryPaths = new HashMap<>(); + + // go through each property + for (String propertyName : stringPropertyNames()) { + // determine if the property is a file repository path + if (StringUtils.startsWith(propertyName, PROVENANCE_REPO_DIRECTORY_PREFIX)) { + // get the repository key + final String key = StringUtils.substringAfter(propertyName, PROVENANCE_REPO_DIRECTORY_PREFIX); + + // attempt to resolve the path specified + provenanceRepositoryPaths.put(key, Paths.get(getProperty(propertyName))); + } + } + return provenanceRepositoryPaths; + } + + public int getMaxFlowFilesPerClaim() { + try { + return Integer.parseInt(getProperty(MAX_FLOWFILES_PER_CLAIM)); + } catch (NumberFormatException nfe) { + return DEFAULT_MAX_FLOWFILES_PER_CLAIM; + } + } + + public String getMaxAppendableClaimSize() { + return getProperty(MAX_APPENDABLE_CLAIM_SIZE); + } + + @Override + public String getProperty(final String key, final String defaultValue) { + final String value = super.getProperty(key, defaultValue); + if (value == null) { + return null; + } + + if (value.trim().isEmpty()) { + return defaultValue; + } + return value; + } + +} diff --git a/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java b/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java new file mode 100644 index 0000000000..aa6f8f305f --- /dev/null +++ b/commons/nifi-properties/src/main/java/org/apache/nifi/util/StringUtils.java @@ -0,0 +1,66 @@ +/* + * 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.util; + +/** + * String Utils based on the Apache Commons Lang String Utils. + * These simple util methods here allow us to avoid a dependency in the core + */ +public class StringUtils { + + public static final String EMPTY = ""; + + public static boolean isBlank(final String str) { + if (str == null || str.isEmpty()) { + return true; + } + for (int i = 0; i < str.length(); i++) { + if (!Character.isWhitespace(str.charAt(i))) { + return false; + } + } + return true; + } + + public static boolean isEmpty(final String str) { + return str == null || str.isEmpty(); + } + + public static boolean startsWith(final String str, final String prefix) { + if (str == null || prefix == null) { + return (str == null && prefix == null); + } + if (prefix.length() > str.length()) { + return false; + } + return str.regionMatches(false, 0, prefix, 0, prefix.length()); + } + + public static String substringAfter(final String str, final String separator) { + if (isEmpty(str)) { + return str; + } + if (separator == null) { + return EMPTY; + } + int pos = str.indexOf(separator); + if (pos == -1) { + return EMPTY; + } + return str.substring(pos + separator.length()); + } +} diff --git a/commons/nifi-security-utils/pom.xml b/commons/nifi-security-utils/pom.xml new file mode 100644 index 0000000000..76e9ac16a4 --- /dev/null +++ b/commons/nifi-security-utils/pom.xml @@ -0,0 +1,40 @@ + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-security-utils + 0.0.1-SNAPSHOT + NiFi Security Utils + Contains security functionality. + + + + org.slf4j + slf4j-api + + + org.apache.commons + commons-lang3 + 3.3.2 + + + + diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java new file mode 100644 index 0000000000..087d891d29 --- /dev/null +++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/CertificateUtils.java @@ -0,0 +1,158 @@ +/* + * 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.security.util; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.net.URL; +import java.security.KeyStore; +import java.security.cert.CertificateParsingException; +import java.security.cert.X509Certificate; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public final class CertificateUtils { + + private static final Logger logger = LoggerFactory.getLogger(CertificateUtils.class); + + /** + * Returns true if the given keystore can be loaded using the given keystore + * type and password. Returns false otherwise. + * @param keystore + * @param keystoreType + * @param password + * @return + */ + public static boolean isStoreValid(final URL keystore, final KeystoreType keystoreType, final char[] password) { + + if (keystore == null) { + throw new IllegalArgumentException("keystore may not be null"); + } else if (keystoreType == null) { + throw new IllegalArgumentException("keystore type may not be null"); + } else if (password == null) { + throw new IllegalArgumentException("password may not be null"); + } + + BufferedInputStream bis = null; + final KeyStore ks; + try { + + // load the keystore + bis = new BufferedInputStream(keystore.openStream()); + ks = KeyStore.getInstance(keystoreType.name()); + ks.load(bis, password); + + return true; + + } catch (Exception e) { + return false; + } finally { + if (bis != null) { + try { + bis.close(); + } catch (final IOException ioe) { + logger.warn("Failed to close input stream", ioe); + } + } + } + } + + /** + * Extracts the username from the specified DN. If the username cannot be + * extracted because the CN is in an unrecognized format, the entire CN is + * returned. If the CN cannot be extracted because the DN is in an + * unrecognized format, the entire DN is returned. + * + * @param dn + * @return + */ + public static String extractUsername(String dn) { + String username = dn; + String cn = ""; + + // ensure the dn is specified + if (StringUtils.isNotBlank(dn)) { + + // attempt to locate the cn + if (dn.startsWith("CN=")) { + cn = StringUtils.substringBetween(dn, "CN=", ","); + } else if (dn.startsWith("/CN=")) { + cn = StringUtils.substringBetween(dn, "CN=", "/"); + } else if (dn.startsWith("C=") || dn.startsWith("/C=")) { + cn = StringUtils.substringAfter(dn, "CN="); + } else if (dn.startsWith("/") && StringUtils.contains(dn, "CN=")) { + cn = StringUtils.substringAfter(dn, "CN="); + } + + // attempt to get the username from the cn + if (StringUtils.isNotBlank(cn)) { + if (cn.endsWith(")")) { + username = StringUtils.substringBetween(cn, "(", ")"); + } else if (cn.contains(" ")) { + username = StringUtils.substringAfterLast(cn, " "); + } else { + username = cn; + } + } + } + + return username; + } + + /** + * Returns a list of subject alternative names. Any name that is represented + * as a String by X509Certificate.getSubjectAlternativeNames() is converted + * to lowercase and returned. + * + * @param certificate a certificate + * @return a list of subject alternative names; list is never null + * @throws CertificateParsingException if parsing the certificate failed + */ + public static List getSubjectAlternativeNames(final X509Certificate certificate) throws CertificateParsingException { + + final Collection> altNames = certificate.getSubjectAlternativeNames(); + if (altNames == null) { + return new ArrayList<>(); + } + + final List result = new ArrayList<>(); + for (final List generalName : altNames) { + /* + * generalName has the name type as the first element a String or + * byte array for the second element. We return any general names + * that are String types. + * + * We don't inspect the numeric name type because some certificates + * incorrectly put IPs and DNS names under the wrong name types. + */ + final Object value = generalName.get(1); + if (value instanceof String) { + result.add(((String) value).toLowerCase()); + } + + } + + return result; + } + + private CertificateUtils() { + } +} diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java new file mode 100644 index 0000000000..741fdde792 --- /dev/null +++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/EncryptionMethod.java @@ -0,0 +1,84 @@ +/* + * 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.security.util; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * Enumeration capturing essential information about the various encryption + * methods that might be supported. + * + * @author none + */ +public enum EncryptionMethod { + + MD5_128AES("PBEWITHMD5AND128BITAES-CBC-OPENSSL", "BC", false), + MD5_256AES("PBEWITHMD5AND256BITAES-CBC-OPENSSL", "BC", false), + SHA1_RC2("PBEWITHSHA1ANDRC2", "BC", false), + SHA1_DES("PBEWITHSHA1ANDDES", "BC", false), + MD5_192AES("PBEWITHMD5AND192BITAES-CBC-OPENSSL", "BC", false), + MD5_DES("PBEWITHMD5ANDDES", "BC", false), + MD5_RC2("PBEWITHMD5ANDRC2", "BC", false), + SHA_192AES("PBEWITHSHAAND192BITAES-CBC-BC", "BC", true), + SHA_40RC4("PBEWITHSHAAND40BITRC4", "BC", true), + SHA256_128AES("PBEWITHSHA256AND128BITAES-CBC-BC", "BC", true), + SHA_128RC2("PBEWITHSHAAND128BITRC2-CBC", "BC", true), + SHA_128AES("PBEWITHSHAAND128BITAES-CBC-BC", "BC", true), + SHA256_192AES("PBEWITHSHA256AND192BITAES-CBC-BC", "BC", true), + SHA_2KEYTRIPLEDES("PBEWITHSHAAND2-KEYTRIPLEDES-CBC", "BC", true), + SHA256_256AES("PBEWITHSHA256AND256BITAES-CBC-BC", "BC", true), + SHA_40RC2("PBEWITHSHAAND40BITRC2-CBC", "BC", true), + SHA_256AES("PBEWITHSHAAND256BITAES-CBC-BC", "BC", true), + SHA_3KEYTRIPLEDES("PBEWITHSHAAND3-KEYTRIPLEDES-CBC", "BC", true), + SHA_TWOFISH("PBEWITHSHAANDTWOFISH-CBC", "BC", true), + SHA_128RC4("PBEWITHSHAAND128BITRC4", "BC", true); + private final String algorithm; + private final String provider; + private final boolean unlimitedStrength; + + EncryptionMethod(String algorithm, String provider, boolean unlimitedStrength) { + this.algorithm = algorithm; + this.provider = provider; + this.unlimitedStrength = unlimitedStrength; + } + + public String getProvider() { + return provider; + } + + public String getAlgorithm() { + return algorithm; + } + + /** + * @return true if algorithm requires unlimited strength policies + */ + public boolean isUnlimitedStrength() { + return unlimitedStrength; + } + + @Override + public String toString() { + final ToStringBuilder builder = new ToStringBuilder(this); + ToStringBuilder.setDefaultStyle(ToStringStyle.SHORT_PREFIX_STYLE); + builder.append("algorithm name", algorithm); + builder.append("Requires unlimited strength JCE policy", unlimitedStrength); + builder.append("Algorithm Provider", provider); + return builder.toString(); + } +} diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.java new file mode 100644 index 0000000000..18574bb0bd --- /dev/null +++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/KeystoreType.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.security.util; + +/** + * Keystore types. + */ +public enum KeystoreType { + + PKCS12, + JKS; +} diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java new file mode 100644 index 0000000000..9abfcc3ccd --- /dev/null +++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SecurityStoreTypes.java @@ -0,0 +1,144 @@ +/* + * 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.security.util; + +import java.io.PrintWriter; +import java.io.Writer; + +/** + * Types of security stores and their related Java system properties. + */ +public enum SecurityStoreTypes { + + TRUSTSTORE( + "javax.net.ssl.trustStore", + "javax.net.ssl.trustStorePassword", + "javax.net.ssl.trustStoreType"), + KEYSTORE( + "javax.net.ssl.keyStore", + "javax.net.ssl.keyStorePassword", + "javax.net.ssl.keyStoreType"); + + /** + * Logs the keystore and truststore Java system property values to the given + * writer. It logPasswords is true, then the keystore and truststore + * password property values are logged. + * + * @param writer a writer to log to + * + * @param logPasswords true if passwords should be logged; false otherwise + */ + public static void logProperties(final Writer writer, + final boolean logPasswords) { + if (writer == null) { + return; + } + + PrintWriter pw = new PrintWriter(writer); + + // keystore properties + pw.println( + KEYSTORE.getStoreProperty() + " = " + System.getProperty(KEYSTORE.getStoreProperty())); + + if (logPasswords) { + pw.println( + KEYSTORE.getStorePasswordProperty() + " = " + + System.getProperty(KEYSTORE.getStoreProperty())); + } + + pw.println( + KEYSTORE.getStoreTypeProperty() + " = " + + System.getProperty(KEYSTORE.getStoreTypeProperty())); + + // truststore properties + pw.println( + TRUSTSTORE.getStoreProperty() + " = " + + System.getProperty(TRUSTSTORE.getStoreProperty())); + + if (logPasswords) { + pw.println( + TRUSTSTORE.getStorePasswordProperty() + " = " + + System.getProperty(TRUSTSTORE.getStoreProperty())); + } + + pw.println( + TRUSTSTORE.getStoreTypeProperty() + " = " + + System.getProperty(TRUSTSTORE.getStoreTypeProperty())); + pw.flush(); + } + + /** + * the Java system property for setting the keystore (or truststore) path + */ + private String storeProperty = ""; + + /** + * the Java system property for setting the keystore (or truststore) + * password + */ + private String storePasswordProperty = ""; + + /** + * the Java system property for setting the keystore (or truststore) type + */ + private String storeTypeProperty = ""; + + /** + * Creates an instance. + * + * @param storeProperty the Java system property for setting the keystore ( + * or truststore) path + * @param storePasswordProperty the Java system property for setting the + * keystore (or truststore) password + * @param storeTypeProperty the Java system property for setting the + * keystore (or truststore) type + */ + SecurityStoreTypes(final String storeProperty, + final String storePasswordProperty, + final String storeTypeProperty) { + this.storeProperty = storeProperty; + this.storePasswordProperty = storePasswordProperty; + this.storeTypeProperty = storeTypeProperty; + } + + /** + * Returns the keystore (or truststore) property. + * + * @return the keystore (or truststore) property + */ + public String getStoreProperty() { + return storeProperty; + } + + /** + * Returns the keystore (or truststore) password property. + * + * @return the keystore (or truststore) password property + */ + public String getStorePasswordProperty() { + return storePasswordProperty; + } + + /** + * Returns the keystore (or truststore) type property. + * + * @return the keystore (or truststore) type property + */ + public String getStoreTypeProperty() { + return storeTypeProperty; + } +} diff --git a/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java new file mode 100644 index 0000000000..2371b0c138 --- /dev/null +++ b/commons/nifi-security-utils/src/main/java/org/apache/nifi/security/util/SslContextFactory.java @@ -0,0 +1,180 @@ +/* + * 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.security.util; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; + +/** + * A factory for creating SSL contexts using the application's security + * properties. + * + * @author unattributed + */ +public final class SslContextFactory { + + public static enum ClientAuth { + + WANT, + REQUIRED, + NONE + } + + /** + * Creates a SSLContext instance using the given information. + * + * @param keystore the full path to the keystore + * @param keystorePasswd the keystore password + * @param keystoreType the type of keystore (e.g., PKCS12, JKS) + * @param truststore the full path to the truststore + * @param truststorePasswd the truststore password + * @param truststoreType the type of truststore (e.g., PKCS12, JKS) + * @param clientAuth the type of client authentication + * + * @return a SSLContext instance + * @throws java.security.KeyStoreException + * @throws java.io.IOException + * @throws java.security.NoSuchAlgorithmException + * @throws java.security.cert.CertificateException + * @throws java.security.UnrecoverableKeyException + * @throws java.security.KeyManagementException + */ + public static SSLContext createSslContext( + final String keystore, final char[] keystorePasswd, final String keystoreType, + final String truststore, final char[] truststorePasswd, final String truststoreType, + final ClientAuth clientAuth) + throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException, + UnrecoverableKeyException, KeyManagementException { + + // prepare the keystore + final KeyStore keyStore = KeyStore.getInstance(keystoreType); + try (final InputStream keyStoreStream = new FileInputStream(keystore)) { + keyStore.load(keyStoreStream, keystorePasswd); + } + final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm()); + keyManagerFactory.init(keyStore, keystorePasswd); + + // prepare the truststore + final KeyStore trustStore = KeyStore.getInstance(truststoreType); + try (final InputStream trustStoreStream = new FileInputStream(truststore)) { + trustStore.load(trustStoreStream, truststorePasswd); + } + final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + trustManagerFactory.init(trustStore); + + // initialize the ssl context + final SSLContext sslContext = SSLContext.getInstance("TLS"); + sslContext.init(keyManagerFactory.getKeyManagers(), trustManagerFactory.getTrustManagers(), new SecureRandom()); + if (ClientAuth.REQUIRED == clientAuth) { + sslContext.getDefaultSSLParameters().setNeedClientAuth(true); + } else if (ClientAuth.WANT == clientAuth) { + sslContext.getDefaultSSLParameters().setWantClientAuth(true); + } else { + sslContext.getDefaultSSLParameters().setWantClientAuth(false); + } + + return sslContext; + + } + + /** + * Creates a SSLContext instance using the given information. + * + * @param keystore the full path to the keystore + * @param keystorePasswd the keystore password + * @param keystoreType the type of keystore (e.g., PKCS12, JKS) + * + * @return a SSLContext instance + * @throws java.security.KeyStoreException + * @throws java.io.IOException + * @throws java.security.NoSuchAlgorithmException + * @throws java.security.cert.CertificateException + * @throws java.security.UnrecoverableKeyException + * @throws java.security.KeyManagementException + */ + public static SSLContext createSslContext( + final String keystore, final char[] keystorePasswd, final String keystoreType) + throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException, + UnrecoverableKeyException, KeyManagementException { + + // prepare the keystore + final KeyStore keyStore = KeyStore.getInstance(keystoreType); + try (final InputStream keyStoreStream = new FileInputStream(keystore)) { + keyStore.load(keyStoreStream, keystorePasswd); + } + final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm()); + keyManagerFactory.init(keyStore, keystorePasswd); + + // initialize the ssl context + final SSLContext ctx = SSLContext.getInstance("TLS"); + ctx.init(keyManagerFactory.getKeyManagers(), new TrustManager[0], new SecureRandom()); + + return ctx; + + } + + /** + * Creates a SSLContext instance using the given information. + * + * @param truststore the full path to the truststore + * @param truststorePasswd the truststore password + * @param truststoreType the type of truststore (e.g., PKCS12, JKS) + * + * @return a SSLContext instance + * @throws java.security.KeyStoreException + * @throws java.io.IOException + * @throws java.security.NoSuchAlgorithmException + * @throws java.security.cert.CertificateException + * @throws java.security.UnrecoverableKeyException + * @throws java.security.KeyManagementException + */ + public static SSLContext createTrustSslContext( + final String truststore, final char[] truststorePasswd, final String truststoreType) + throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException, + UnrecoverableKeyException, KeyManagementException { + + // prepare the truststore + final KeyStore trustStore = KeyStore.getInstance(truststoreType); + try (final InputStream trustStoreStream = new FileInputStream(truststore)) { + trustStore.load(trustStoreStream, truststorePasswd); + } + final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + trustManagerFactory.init(trustStore); + + // initialize the ssl context + final SSLContext ctx = SSLContext.getInstance("TLS"); + ctx.init(new KeyManager[0], trustManagerFactory.getTrustManagers(), new SecureRandom()); + + return ctx; + + } + +} diff --git a/commons/nifi-socket-utils/pom.xml b/commons/nifi-socket-utils/pom.xml new file mode 100644 index 0000000000..8e064339ef --- /dev/null +++ b/commons/nifi-socket-utils/pom.xml @@ -0,0 +1,65 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-socket-utils + 0.0.1-SNAPSHOT + NiFi Socket Utils + Utilities for socket communication + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-logging-utils + 0.0.1-SNAPSHOT + + + commons-net + commons-net + 3.3 + + + org.apache.commons + commons-lang3 + 3.3.2 + + + commons-io + commons-io + 2.4 + + + org.apache.nifi + nifi-properties + 0.0.1-SNAPSHOT + + + diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java new file mode 100644 index 0000000000..172c593995 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/AbstractChannelReader.java @@ -0,0 +1,166 @@ +/* + * 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.io.nio; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.nifi.io.nio.consumer.StreamConsumer; +import org.apache.nifi.io.nio.consumer.StreamConsumerFactory; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * @author none + */ +public abstract class AbstractChannelReader implements Runnable { + + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractChannelReader.class); + private final String uniqueId; + private final SelectionKey key; + private final BufferPool bufferPool; + private final StreamConsumer consumer; + private final AtomicBoolean isClosed = new AtomicBoolean(false); + private final AtomicReference> future = new AtomicReference<>(null);//the future on which this reader runs... + + public AbstractChannelReader(final String id, final SelectionKey key, final BufferPool empties, final StreamConsumerFactory consumerFactory) { + this.uniqueId = id; + this.key = key; + this.bufferPool = empties; + this.consumer = consumerFactory.newInstance(id); + consumer.setReturnBufferQueue(bufferPool); + } + + protected void setScheduledFuture(final ScheduledFuture future) { + this.future.set(future); + } + + protected ScheduledFuture getScheduledFuture() { + return future.get(); + } + + protected SelectionKey getSelectionKey() { + return key; + } + + public boolean isClosed() { + return isClosed.get(); + } + + private void closeStream() { + if (isClosed.get()) { + return; + } + try { + isClosed.set(true); + future.get().cancel(false); + key.cancel(); + key.channel().close(); + } catch (final IOException ioe) { + LOGGER.warn("Unable to cleanly close stream due to " + ioe); + } finally { + consumer.signalEndOfStream(); + } + } + + /** + * Allows a subclass to specifically handle how it reads from the given + * key's channel into the given buffer. + * + * @param key + * @param buffer + * @return the number of bytes read in the final read cycle. A value of zero + * or more indicates the channel is still open but a value of -1 indicates + * end of stream. + * @throws IOException + */ + protected abstract int fillBuffer(SelectionKey key, ByteBuffer buffer) throws IOException; + + @Override + public final void run() { + if (!key.isValid() || consumer.isConsumerFinished()) { + closeStream(); + return; + } + if (!key.isReadable()) { + return;//there is nothing available to read...or we aren't allow to read due to throttling + } + ByteBuffer buffer = null; + try { + buffer = bufferPool.poll(); + if (buffer == null) { + return; // no buffers available - come back later + } + final int bytesRead = fillBuffer(key, buffer); + buffer.flip(); + if (buffer.remaining() > 0) { + consumer.addFilledBuffer(buffer); + buffer = null; //clear the reference - is now the consumer's responsiblity + } else { + buffer.clear(); + bufferPool.returnBuffer(buffer, 0); + buffer = null; //clear the reference - is now back to the queue + } + if (bytesRead < 0) { //we've reached the end + closeStream(); + } + } catch (final Exception ioe) { + closeStream(); + LOGGER.error("Closed channel reader " + this + " due to " + ioe); + } finally { + if (buffer != null) { + buffer.clear(); + bufferPool.returnBuffer(buffer, 0); + } + } + } + + @Override + public final boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + if (obj.getClass() != getClass()) { + return false; + } + AbstractChannelReader rhs = (AbstractChannelReader) obj; + return new EqualsBuilder().appendSuper(super.equals(obj)).append(uniqueId, rhs.uniqueId).isEquals(); + } + + @Override + public final int hashCode() { + return new HashCodeBuilder(17, 37).append(uniqueId).toHashCode(); + } + + @Override + public final String toString() { + return new ToStringBuilder(this, ToStringStyle.NO_FIELD_NAMES_STYLE).append(uniqueId).toString(); + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java new file mode 100644 index 0000000000..a413ad26ab --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/BufferPool.java @@ -0,0 +1,114 @@ +/* + * 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.io.nio; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Calendar; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingDeque; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * @author none + */ +public class BufferPool implements Runnable { + + private static final Logger LOGGER = LoggerFactory.getLogger(BufferPool.class); + final BlockingQueue bufferPool; + private final static double ONE_MB = 1 << 20; + private Calendar lastRateSampleTime = Calendar.getInstance(); + private final Calendar startTime = Calendar.getInstance(); + double lastRateSampleMBps = -1.0; + double overallMBps = -1.0; + private long totalBytesExtracted = 0L; + private long lastTotalBytesExtracted = 0L; + final double maxRateMBps; + + public BufferPool(final int bufferCount, final int bufferCapacity, final boolean allocateDirect, final double maxRateMBps) { + bufferPool = new LinkedBlockingDeque<>(BufferPool.createBuffers(bufferCount, bufferCapacity, allocateDirect)); + this.maxRateMBps = maxRateMBps; + } + + /** + * Returns the given buffer to the pool - and clears it. + * + * @param buffer + * @param bytesProcessed + * @return + */ + public synchronized boolean returnBuffer(ByteBuffer buffer, final int bytesProcessed) { + totalBytesExtracted += bytesProcessed; + buffer.clear(); + return bufferPool.add(buffer); + } + + //here we enforce the desired rate we want by restricting access to buffers when we're over rate + public synchronized ByteBuffer poll() { + computeRate(); + final double weightedAvg = (lastRateSampleMBps * 0.7) + (overallMBps * 0.3); + if (overallMBps >= maxRateMBps || weightedAvg >= maxRateMBps) { + return null; + } + return bufferPool.poll(); + } + + public int size() { + return bufferPool.size(); + } + + private synchronized void computeRate() { + final Calendar now = Calendar.getInstance(); + final long measurementDurationMillis = now.getTimeInMillis() - lastRateSampleTime.getTimeInMillis(); + final double duractionSecs = ((double) measurementDurationMillis) / 1000.0; + if (duractionSecs >= 0.75) { //recompute every 3/4 second or when we're too fast + final long totalDuractionMillis = now.getTimeInMillis() - startTime.getTimeInMillis(); + final double totalDurationSecs = ((double) totalDuractionMillis) / 1000.0; + final long differenceBytes = totalBytesExtracted - lastTotalBytesExtracted; + lastTotalBytesExtracted = totalBytesExtracted; + lastRateSampleTime = now; + final double bps = ((double) differenceBytes) / duractionSecs; + final double totalBps = ((double) totalBytesExtracted / totalDurationSecs); + lastRateSampleMBps = bps / ONE_MB; + overallMBps = totalBps / ONE_MB; + } + } + + public static List createBuffers(final int bufferCount, final int bufferCapacity, final boolean allocateDirect) { + final List buffers = new ArrayList<>(); + for (int i = 0; i < bufferCount; i++) { + final ByteBuffer buffer = (allocateDirect) ? ByteBuffer.allocateDirect(bufferCapacity) : ByteBuffer.allocate(bufferCapacity); + buffers.add(buffer); + } + return buffers; + } + + private void logChannelReadRates() { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug(String.format("Overall rate= %,.4f MB/s / Current Rate= %,.4f MB/s / Total Bytes Read= %d", overallMBps, lastRateSampleMBps, totalBytesExtracted)); + } + } + + @Override + public void run() { + computeRate(); + logChannelReadRates(); + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.java new file mode 100644 index 0000000000..2ae2c07d00 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelDispatcher.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.io.nio; + +import java.io.IOException; +import java.nio.channels.DatagramChannel; +import java.nio.channels.SelectableChannel; +import java.nio.channels.SelectionKey; +import java.nio.channels.Selector; +import java.nio.channels.ServerSocketChannel; +import java.nio.channels.SocketChannel; +import java.util.Iterator; +import java.util.UUID; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.io.nio.consumer.StreamConsumerFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * @author none + */ +public final class ChannelDispatcher implements Runnable { + + private static final Logger LOGGER = LoggerFactory.getLogger(ChannelDispatcher.class); + private final Selector serverSocketSelector; + private final Selector socketChannelSelector; + private final ScheduledExecutorService executor; + private final BufferPool emptyBuffers; + private final StreamConsumerFactory factory; + private final AtomicLong channelReaderFrequencyMilliseconds = new AtomicLong(DEFAULT_CHANNEL_READER_PERIOD_MILLISECONDS); + private final long timeout; + private volatile boolean stop = false; + public static final long DEFAULT_CHANNEL_READER_PERIOD_MILLISECONDS = 100L; + + public ChannelDispatcher(final Selector serverSocketSelector, final Selector socketChannelSelector, final ScheduledExecutorService service, + final StreamConsumerFactory factory, final BufferPool buffers, final long timeout, final TimeUnit unit) { + this.serverSocketSelector = serverSocketSelector; + this.socketChannelSelector = socketChannelSelector; + this.executor = service; + this.factory = factory; + emptyBuffers = buffers; + this.timeout = TimeUnit.MILLISECONDS.convert(timeout, unit); + } + + public void setChannelReaderFrequency(final long period, final TimeUnit timeUnit) { + channelReaderFrequencyMilliseconds.set(TimeUnit.MILLISECONDS.convert(period, timeUnit)); + } + + @Override + public void run() { + while (!stop) { + try { + selectServerSocketKeys(); + selectSocketChannelKeys(); + } catch (final Exception ex) { + LOGGER.warn("Key selection failed: {} Normal during shutdown.", new Object[]{ex}); + } + } + } + + /* + * When serverSocketsChannels are registered with the selector, want each invoke of this method to loop through all + * channels' keys. + * + * @throws IOException + */ + private void selectServerSocketKeys() throws IOException { + int numSelected = serverSocketSelector.select(timeout); + if (numSelected == 0) { + return; + } + + // for each registered server socket - see if any connections are waiting to be established + final Iterator itr = serverSocketSelector.selectedKeys().iterator(); + while (itr.hasNext()) { + SelectionKey serverSocketkey = itr.next(); + final SelectableChannel channel = serverSocketkey.channel(); + AbstractChannelReader reader = null; + if (serverSocketkey.isValid() && serverSocketkey.isAcceptable()) { + final ServerSocketChannel ssChannel = (ServerSocketChannel) serverSocketkey.channel(); + final SocketChannel sChannel = ssChannel.accept(); + if (sChannel != null) { + sChannel.configureBlocking(false); + final SelectionKey socketChannelKey = sChannel.register(socketChannelSelector, SelectionKey.OP_READ); + final String readerId = sChannel.socket().toString(); + reader = new SocketChannelReader(readerId, socketChannelKey, emptyBuffers, factory); + final ScheduledFuture readerFuture = executor.scheduleWithFixedDelay(reader, 10L, + channelReaderFrequencyMilliseconds.get(), TimeUnit.MILLISECONDS); + reader.setScheduledFuture(readerFuture); + socketChannelKey.attach(reader); + } + } + itr.remove(); // do this so that the next select operation returns a positive value; otherwise, it will return 0. + if (reader != null && LOGGER.isDebugEnabled()) { + LOGGER.debug(this + " New Connection established. Server channel: " + channel + " Reader: " + reader); + } + } + } + + /* + * When invoking this method, only want to iterate through the selected keys once. When a key is entered into the selectors + * selected key set, select will return a positive value. The next select will return 0 if nothing has changed. Note that + * the selected key set is not manually changed via a remove operation. + * + * @throws IOException + */ + private void selectSocketChannelKeys() throws IOException { + // once a channel associated with a key in this selector is 'ready', it causes this select to immediately return. + // thus, for each trip through the run() we only get hit with one real timeout...the one in selectServerSocketKeys. + int numSelected = socketChannelSelector.select(timeout); + if (numSelected == 0) { + return; + } + + for (SelectionKey socketChannelKey : socketChannelSelector.selectedKeys()) { + final SelectableChannel channel = socketChannelKey.channel(); + AbstractChannelReader reader = null; + // there are 2 kinds of channels in this selector, both which have their own readers and are executed in their own + // threads. We will get here whenever a new SocketChannel is created due to an incoming connection. However, + // for a DatagramChannel we don't want to create a new reader unless it is a new DatagramChannel. The only + // way to tell if it's new is the lack of an attachment. + if (channel instanceof DatagramChannel && socketChannelKey.attachment() == null) { + reader = new DatagramChannelReader(UUID.randomUUID().toString(), socketChannelKey, emptyBuffers, factory); + socketChannelKey.attach(reader); + final ScheduledFuture readerFuture = executor.scheduleWithFixedDelay(reader, 10L, channelReaderFrequencyMilliseconds.get(), + TimeUnit.MILLISECONDS); + reader.setScheduledFuture(readerFuture); + } + if (reader != null && LOGGER.isDebugEnabled()) { + LOGGER.debug(this + " New Connection established. Server channel: " + channel + " Reader: " + reader); + } + } + + } + + public void stop() { + stop = true; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java new file mode 100644 index 0000000000..b0a1cfba2d --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/ChannelListener.java @@ -0,0 +1,228 @@ +/* + * 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.io.nio; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.net.StandardSocketOptions; +import java.nio.channels.DatagramChannel; +import java.nio.channels.SelectionKey; +import java.nio.channels.Selector; +import java.nio.channels.ServerSocketChannel; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.io.nio.consumer.StreamConsumerFactory; + +import org.apache.commons.io.IOUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class provides the entry point to NIO based socket listeners for NiFi + * processors and services. There are 2 supported types of Listeners, Datagram + * (UDP based transmissions) and ServerSocket (TCP based transmissions). This + * will create the ChannelDispatcher, which is a Runnable and is controlled via + * the ScheduledExecutorService, which is also created by this class. The + * ChannelDispatcher handles connections to the ServerSocketChannels and creates + * the readers associated with the resulting SocketChannels. Additionally, this + * creates and manages two Selectors, one for ServerSocketChannels and another + * for SocketChannels and DatagramChannels. + * + * The threading model for this consists of one thread for the + * ChannelDispatcher, one thread per added SocketChannel reader, one thread per + * added DatagramChannel reader. The ChannelDispatcher is not scheduled with + * fixed delay as the others are. It is throttled by the provided timeout value. + * Within the ChannelDispatcher there are two blocking operations which will + * block for the given timeout each time through the enclosing loop. + * + * All channels are cached in one of the two Selectors via their SelectionKey. + * The serverSocketSelector maintains all the added ServerSocketChannels; the + * socketChannelSelector maintains the all the add DatagramChannels and the + * created SocketChannels. Further, the SelectionKey of the DatagramChannel and + * the SocketChannel is injected with the channel's associated reader. + * + * All ChannelReaders will get throttled by the unavailability of buffers in the + * provided BufferPool. This is designed to create back pressure. + * + * @author none + */ +public final class ChannelListener { + + private static final Logger LOGGER = LoggerFactory.getLogger(ChannelListener.class); + private final ScheduledExecutorService executor; + private final Selector serverSocketSelector; // used to listen for new connections + private final Selector socketChannelSelector; // used to listen on existing connections + private final ChannelDispatcher channelDispatcher; + private final BufferPool bufferPool; + private final int initialBufferPoolSize; + private volatile long channelReaderFrequencyMSecs = 50; + + public ChannelListener(final int threadPoolSize, final StreamConsumerFactory consumerFactory, final BufferPool bufferPool, int timeout, + TimeUnit unit) throws IOException { + this.executor = Executors.newScheduledThreadPool(threadPoolSize + 1); // need to allow for long running ChannelDispatcher thread + this.serverSocketSelector = Selector.open(); + this.socketChannelSelector = Selector.open(); + this.bufferPool = bufferPool; + this.initialBufferPoolSize = bufferPool.size(); + channelDispatcher = new ChannelDispatcher(serverSocketSelector, socketChannelSelector, executor, consumerFactory, bufferPool, + timeout, unit); + executor.schedule(channelDispatcher, 50, TimeUnit.MILLISECONDS); + } + + public void setChannelReaderSchedulingPeriod(final long period, final TimeUnit unit) { + channelReaderFrequencyMSecs = TimeUnit.MILLISECONDS.convert(period, unit); + channelDispatcher.setChannelReaderFrequency(period, unit); + } + + /** + * Adds a server socket channel for listening to connections. + * + * @param nicIPAddress - if null binds to wildcard address + * @param port - port to bind to + * @param receiveBufferSize - size of OS receive buffer to request. If less + * than 0 then will not be set and OS default will win. + * @throws IOException + */ + public void addServerSocket(final InetAddress nicIPAddress, final int port, final int receiveBufferSize) + throws IOException { + final ServerSocketChannel ssChannel = ServerSocketChannel.open(); + ssChannel.configureBlocking(false); + if (receiveBufferSize > 0) { + ssChannel.setOption(StandardSocketOptions.SO_RCVBUF, receiveBufferSize); + final int actualReceiveBufSize = ssChannel.getOption(StandardSocketOptions.SO_RCVBUF); + if (actualReceiveBufSize < receiveBufferSize) { + LOGGER.warn(this + " attempted to set TCP Receive Buffer Size to " + + receiveBufferSize + " bytes but could only set to " + actualReceiveBufSize + + "bytes. You may want to consider changing the Operating System's " + + "maximum receive buffer"); + } + } + ssChannel.setOption(StandardSocketOptions.SO_REUSEADDR, true); + ssChannel.bind(new InetSocketAddress(nicIPAddress, port)); + ssChannel.register(serverSocketSelector, SelectionKey.OP_ACCEPT); + } + + /** + * Binds to listen for data grams on the given local IPAddress/port + * + * @param nicIPAddress - if null will listen on wildcard address, which + * means datagrams will be received on all local network interfaces. + * Otherwise, will bind to the provided IP address associated with some NIC. + * @param port - the port to listen on + * @param receiveBufferSize - the number of bytes to request for a receive + * buffer from OS + * @throws IOException + */ + public void addDatagramChannel(final InetAddress nicIPAddress, final int port, final int receiveBufferSize) + throws IOException { + final DatagramChannel dChannel = createAndBindDatagramChannel(nicIPAddress, port, receiveBufferSize); + dChannel.register(socketChannelSelector, SelectionKey.OP_READ); + } + + /** + * Binds to listen for data grams on the given local IPAddress/port and + * restricts receipt of datagrams to those from the provided host and port, + * must specify both. This improves performance for datagrams coming from a + * sender that is known a-priori. + * + * @param nicIPAddress - if null will listen on wildcard address, which + * means datagrams will be received on all local network interfaces. + * Otherwise, will bind to the provided IP address associated with some NIC. + * @param port - the port to listen on. This is used to provide a well-known + * destination for a sender. + * @param receiveBufferSize - the number of bytes to request for a receive + * buffer from OS + * @param sendingHost - the hostname, or IP address, of the sender of + * datagrams. Only datagrams from this host will be received. If this is + * null the wildcard ip is used, which means datagrams may be received from + * any network interface on the local host. + * @param sendingPort - the port used by the sender of datagrams. Only + * datagrams from this port will be received. + * @throws IOException + */ + public void addDatagramChannel(final InetAddress nicIPAddress, final int port, final int receiveBufferSize, final String sendingHost, + final Integer sendingPort) throws IOException { + + if (sendingHost == null || sendingPort == null) { + addDatagramChannel(nicIPAddress, port, receiveBufferSize); + return; + } + final DatagramChannel dChannel = createAndBindDatagramChannel(nicIPAddress, port, receiveBufferSize); + dChannel.connect(new InetSocketAddress(sendingHost, sendingPort)); + dChannel.register(socketChannelSelector, SelectionKey.OP_READ); + } + + private DatagramChannel createAndBindDatagramChannel(final InetAddress nicIPAddress, final int port, final int receiveBufferSize) + throws IOException { + final DatagramChannel dChannel = DatagramChannel.open(); + dChannel.configureBlocking(false); + if (receiveBufferSize > 0) { + dChannel.setOption(StandardSocketOptions.SO_RCVBUF, receiveBufferSize); + final int actualReceiveBufSize = dChannel.getOption(StandardSocketOptions.SO_RCVBUF); + if (actualReceiveBufSize < receiveBufferSize) { + LOGGER.warn(this + " attempted to set UDP Receive Buffer Size to " + + receiveBufferSize + " bytes but could only set to " + actualReceiveBufSize + + "bytes. You may want to consider changing the Operating System's " + + "maximum receive buffer"); + } + } + dChannel.setOption(StandardSocketOptions.SO_REUSEADDR, true); + dChannel.bind(new InetSocketAddress(nicIPAddress, port)); + return dChannel; + } + + public void shutdown(final long period, final TimeUnit timeUnit) { + channelDispatcher.stop(); + for (SelectionKey selectionKey : socketChannelSelector.keys()) { + final AbstractChannelReader reader = (AbstractChannelReader) selectionKey.attachment(); + selectionKey.cancel(); + if (reader != null) { + while (!reader.isClosed()) { + try { + Thread.sleep(channelReaderFrequencyMSecs); + } catch (InterruptedException e) { + } + } + final ScheduledFuture readerFuture = reader.getScheduledFuture(); + readerFuture.cancel(false); + } + IOUtils.closeQuietly(selectionKey.channel()); // should already be closed via reader, but if reader did not exist... + } + IOUtils.closeQuietly(socketChannelSelector); + + for (SelectionKey selectionKey : serverSocketSelector.keys()) { + selectionKey.cancel(); + IOUtils.closeQuietly(selectionKey.channel()); + } + IOUtils.closeQuietly(serverSocketSelector); + executor.shutdown(); + try { + executor.awaitTermination(period, timeUnit); + } catch (final InterruptedException ex) { + LOGGER.warn("Interrupted while trying to shutdown executor"); + } + final int currentBufferPoolSize = bufferPool.size(); + final String warning = (currentBufferPoolSize != initialBufferPoolSize) ? "Initial buffer count=" + initialBufferPoolSize + + " Current buffer count=" + currentBufferPoolSize + + " Could indicate a buffer leak. Ensure all consumers are executed until they complete." : ""; + LOGGER.info("Channel listener shutdown. " + warning); + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java new file mode 100644 index 0000000000..1eb5c7e3c9 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/DatagramChannelReader.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io.nio; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.DatagramChannel; +import java.nio.channels.SelectionKey; + +import org.apache.nifi.io.nio.consumer.StreamConsumerFactory; + +/** + * + * @author none + */ +public final class DatagramChannelReader extends AbstractChannelReader { + + public static final int MAX_UDP_PACKET_SIZE = 65507; + + public DatagramChannelReader(final String id, final SelectionKey key, final BufferPool empties, final StreamConsumerFactory consumerFactory) { + super(id, key, empties, consumerFactory); + } + + /** + * Will receive UDP data from channel and won't receive anything unless the + * given buffer has enough space for at least one full max udp packet. + * + * @param key + * @param buffer + * @return + * @throws IOException + */ + @Override + protected int fillBuffer(final SelectionKey key, final ByteBuffer buffer) throws IOException { + final DatagramChannel dChannel = (DatagramChannel) key.channel(); + final int initialBufferPosition = buffer.position(); + while (buffer.remaining() > MAX_UDP_PACKET_SIZE && key.isValid() && key.isReadable()) { + if (dChannel.receive(buffer) == null) { + break; + } + } + return buffer.position() - initialBufferPosition; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java new file mode 100644 index 0000000000..db2c102cbd --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/SocketChannelReader.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io.nio; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.SelectionKey; +import java.nio.channels.SocketChannel; +import org.apache.nifi.io.nio.consumer.StreamConsumerFactory; + +/** + * + * @author none + */ +public final class SocketChannelReader extends AbstractChannelReader { + + public SocketChannelReader(final String id, final SelectionKey key, final BufferPool empties, final StreamConsumerFactory consumerFactory) { + super(id, key, empties, consumerFactory); + } + + /** + * Receives TCP data from the socket channel for the given key. + * + * @param key + * @param buffer + * @return + * @throws IOException + */ + @Override + protected int fillBuffer(final SelectionKey key, final ByteBuffer buffer) throws IOException { + int bytesRead = 0; + final SocketChannel sChannel = (SocketChannel) key.channel(); + while (key.isValid() && key.isReadable()) { + bytesRead = sChannel.read(buffer); + if (bytesRead <= 0) { + break; + } + } + return bytesRead; + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java new file mode 100644 index 0000000000..fce59c60ab --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/AbstractStreamConsumer.java @@ -0,0 +1,132 @@ +/* + * 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.io.nio.consumer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.nifi.io.nio.BufferPool; +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +/** + * + * @author none + */ +public abstract class AbstractStreamConsumer implements StreamConsumer { + + private final String uniqueId; + private BufferPool bufferPool = null; + private final BlockingQueue filledBuffers = new LinkedBlockingQueue<>(); + private final AtomicBoolean streamEnded = new AtomicBoolean(false); + private final AtomicBoolean consumerEnded = new AtomicBoolean(false); + + public AbstractStreamConsumer(final String id) { + uniqueId = id; + } + + @Override + public final void setReturnBufferQueue(final BufferPool returnQueue) { + bufferPool = returnQueue; + } + + @Override + public final void addFilledBuffer(final ByteBuffer buffer) { + if (isConsumerFinished()) { + buffer.clear(); + bufferPool.returnBuffer(buffer, buffer.remaining()); + } else { + filledBuffers.add(buffer); + } + } + + @Override + public final void process() throws IOException { + if (isConsumerFinished()) { + return; + } + if (streamEnded.get() && filledBuffers.isEmpty()) { + consumerEnded.set(true); + onConsumerDone(); + return; + } + final ByteBuffer buffer = filledBuffers.poll(); + if (buffer != null) { + final int bytesToProcess = buffer.remaining(); + try { + processBuffer(buffer); + } finally { + buffer.clear(); + bufferPool.returnBuffer(buffer, bytesToProcess); + } + } + } + + protected abstract void processBuffer(ByteBuffer buffer) throws IOException; + + @Override + public final void signalEndOfStream() { + streamEnded.set(true); + } + + /** + * Convenience method that is called when the consumer is done processing + * based on being told the signal is end of stream and has processed all + * given buffers. + */ + protected void onConsumerDone() { + } + + @Override + public final boolean isConsumerFinished() { + return consumerEnded.get(); + } + + @Override + public final String getId() { + return uniqueId; + } + + @Override + public final boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (obj == this) { + return true; + } + if (obj.getClass() != getClass()) { + return false; + } + AbstractStreamConsumer rhs = (AbstractStreamConsumer) obj; + return new EqualsBuilder().appendSuper(super.equals(obj)).append(uniqueId, rhs.uniqueId).isEquals(); + } + + @Override + public final int hashCode() { + return new HashCodeBuilder(19, 23).append(uniqueId).toHashCode(); + } + + @Override + public final String toString() { + return new ToStringBuilder(this, ToStringStyle.NO_FIELD_NAMES_STYLE).append(uniqueId).toString(); + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java new file mode 100644 index 0000000000..d75b7d7a0b --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumer.java @@ -0,0 +1,80 @@ +/* + * 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.io.nio.consumer; + +import java.io.IOException; +import java.nio.ByteBuffer; +import org.apache.nifi.io.nio.BufferPool; + +/** + * A StreamConsumer must be thread safe. It may be accessed concurrently by a + * thread providing data to process and another thread that is processing that + * data. + * + * @author none + */ +public interface StreamConsumer { + + /** + * Will be called once just after construction. It provides the queue to + * which processed and emptied and cleared buffers must be returned. For + * each time addFilledBuffer is called there should be an + * associated add to this given queue. If not, buffers will run out and all + * stream processing will halt. READ THIS!!! + * + * @param returnQueue + */ + void setReturnBufferQueue(BufferPool returnQueue); + + /** + * Will be called by the thread that produces byte buffers with available + * data to be processed. If the consumer is finished this should simply + * return the given buffer to the return buffer queue (after it is cleared) + * + * @param buffer + */ + void addFilledBuffer(ByteBuffer buffer); + + /** + * Will be called by the thread that executes the consumption of data. May + * be called many times though once isConsumerFinished returns + * true this method will likely do nothing. + * @throws java.io.IOException + */ + void process() throws IOException; + + /** + * Called once the end of the input stream is detected + */ + void signalEndOfStream(); + + /** + * If true signals the consumer is done consuming data and will not process + * any more buffers. + * + * @return + */ + boolean isConsumerFinished(); + + /** + * Uniquely identifies the consumer + * + * @return + */ + String getId(); + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.java new file mode 100644 index 0000000000..df298d57ef --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/nio/consumer/StreamConsumerFactory.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.io.nio.consumer; + +/** + * + * @author none + */ +public interface StreamConsumerFactory { + + StreamConsumer newInstance(String streamId); + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java new file mode 100644 index 0000000000..7ed5ad4313 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SSLContextFactory.java @@ -0,0 +1,102 @@ +/* + * 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.io.socket; + +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; + +import org.apache.nifi.util.NiFiProperties; + +public class SSLContextFactory { + + private final String keystore; + private final char[] keystorePass; + private final String keystoreType; + private final String truststore; + private final char[] truststorePass; + private final String truststoreType; + + private final KeyManager[] keyManagers; + private final TrustManager[] trustManagers; + + public SSLContextFactory(final NiFiProperties properties) throws NoSuchAlgorithmException, CertificateException, FileNotFoundException, IOException, KeyStoreException, UnrecoverableKeyException { + keystore = properties.getProperty(NiFiProperties.SECURITY_KEYSTORE); + keystorePass = getPass(properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_PASSWD)); + keystoreType = properties.getProperty(NiFiProperties.SECURITY_KEYSTORE_TYPE); + + truststore = properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE); + truststorePass = getPass(properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_PASSWD)); + truststoreType = properties.getProperty(NiFiProperties.SECURITY_TRUSTSTORE_TYPE); + + // prepare the keystore + final KeyStore keyStore = KeyStore.getInstance(keystoreType); + keyStore.load(new FileInputStream(keystore), keystorePass); + final KeyManagerFactory keyManagerFactory = KeyManagerFactory.getInstance(KeyManagerFactory.getDefaultAlgorithm()); + keyManagerFactory.init(keyStore, keystorePass); + + // prepare the truststore + final KeyStore trustStore = KeyStore.getInstance(truststoreType); + trustStore.load(new FileInputStream(truststore), truststorePass); + final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + trustManagerFactory.init(trustStore); + + keyManagers = keyManagerFactory.getKeyManagers(); + trustManagers = trustManagerFactory.getTrustManagers(); + } + + private static char[] getPass(final String password) { + return password == null ? null : password.toCharArray(); + } + + /** + * Creates a SSLContext instance using the given information. + * + * + * @return a SSLContext instance + * @throws java.security.KeyStoreException + * @throws java.io.IOException + * @throws java.security.NoSuchAlgorithmException + * @throws java.security.cert.CertificateException + * @throws java.security.UnrecoverableKeyException + * @throws java.security.KeyManagementException + */ + public SSLContext createSslContext() throws KeyStoreException, IOException, NoSuchAlgorithmException, CertificateException, + UnrecoverableKeyException, KeyManagementException { + + // initialize the ssl context + final SSLContext sslContext = SSLContext.getInstance("TLS"); + sslContext.init(keyManagers, trustManagers, new SecureRandom()); + sslContext.getDefaultSSLParameters().setNeedClientAuth(true); + + return sslContext; + + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java new file mode 100644 index 0000000000..fc279fbc3c --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/ServerSocketConfiguration.java @@ -0,0 +1,83 @@ +/* + * 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.io.socket; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; + +import javax.net.ssl.SSLContext; + +/** + * @author unattributed + */ +public final class ServerSocketConfiguration { + + private boolean needClientAuth; + private Integer socketTimeout; + private Boolean reuseAddress; + private Integer receiveBufferSize; + private SSLContextFactory sslContextFactory; + + public ServerSocketConfiguration() { + } + + public SSLContext createSSLContext() throws KeyManagementException, NoSuchAlgorithmException, UnrecoverableKeyException, KeyStoreException, CertificateException, FileNotFoundException, IOException { + return sslContextFactory == null ? null : sslContextFactory.createSslContext(); + } + + public void setSSLContextFactory(final SSLContextFactory sslContextFactory) { + this.sslContextFactory = sslContextFactory; + } + + public Integer getSocketTimeout() { + return socketTimeout; + } + + public void setSocketTimeout(Integer socketTimeout) { + this.socketTimeout = socketTimeout; + } + + public boolean getNeedClientAuth() { + return needClientAuth; + } + + public void setNeedClientAuth(boolean needClientAuth) { + this.needClientAuth = needClientAuth; + } + + public Boolean getReuseAddress() { + return reuseAddress; + } + + public void setReuseAddress(Boolean reuseAddress) { + this.reuseAddress = reuseAddress; + } + + public Integer getReceiveBufferSize() { + return receiveBufferSize; + } + + public void setReceiveBufferSize(Integer receiveBufferSize) { + this.receiveBufferSize = receiveBufferSize; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java new file mode 100644 index 0000000000..c24b540cba --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketConfiguration.java @@ -0,0 +1,116 @@ +/* + * 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.io.socket; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; + +import javax.net.ssl.SSLContext; + +/** + * @author unattributed + */ +public final class SocketConfiguration { + + private Integer socketTimeout; + private Integer receiveBufferSize; + private Integer sendBufferSize; + private Boolean reuseAddress; + private Boolean keepAlive; + private Boolean oobInline; + private Boolean tcpNoDelay; + private Integer trafficClass; + private SSLContextFactory sslContextFactory; + + public SSLContext createSSLContext() throws KeyManagementException, NoSuchAlgorithmException, UnrecoverableKeyException, KeyStoreException, CertificateException, FileNotFoundException, IOException { + return sslContextFactory == null ? null : sslContextFactory.createSslContext(); + } + + public void setSSLContextFactory(final SSLContextFactory sslContextFactory) { + this.sslContextFactory = sslContextFactory; + } + + public Integer getSocketTimeout() { + return socketTimeout; + } + + public void setSocketTimeout(Integer socketTimeout) { + this.socketTimeout = socketTimeout; + } + + public Boolean getReuseAddress() { + return reuseAddress; + } + + public void setReuseAddress(Boolean reuseAddress) { + this.reuseAddress = reuseAddress; + } + + public Boolean getKeepAlive() { + return keepAlive; + } + + public void setKeepAlive(Boolean keepAlive) { + this.keepAlive = keepAlive; + } + + public Boolean getOobInline() { + return oobInline; + } + + public void setOobInline(Boolean oobInline) { + this.oobInline = oobInline; + } + + public Integer getReceiveBufferSize() { + return receiveBufferSize; + } + + public void setReceiveBufferSize(Integer receiveBufferSize) { + this.receiveBufferSize = receiveBufferSize; + } + + public Integer getSendBufferSize() { + return sendBufferSize; + } + + public void setSendBufferSize(Integer sendBufferSize) { + this.sendBufferSize = sendBufferSize; + } + + public Boolean getTcpNoDelay() { + return tcpNoDelay; + } + + public void setTcpNoDelay(Boolean tcpNoDelay) { + this.tcpNoDelay = tcpNoDelay; + } + + public Integer getTrafficClass() { + return trafficClass; + } + + public void setTrafficClass(Integer trafficClass) { + this.trafficClass = trafficClass; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java new file mode 100644 index 0000000000..e02791af07 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketListener.java @@ -0,0 +1,211 @@ +/* + * 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.io.socket; + +import java.io.IOException; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.logging.NiFiLog; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements a listener for TCP/IP messages sent over unicast socket. + * + * @author unattributed + */ +public abstract class SocketListener { + + private static final int DEFAULT_SHUTDOWN_LISTENER_SECONDS = 5; + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketListener.class)); + private volatile ExecutorService executorService; // volatile to guarantee most current value is visible + private volatile ServerSocket serverSocket; // volatile to guarantee most current value is visible + private final int numThreads; + private final int port; + private final ServerSocketConfiguration configuration; + private final AtomicInteger shutdownListenerSeconds = new AtomicInteger(DEFAULT_SHUTDOWN_LISTENER_SECONDS); + + public SocketListener( + final int numThreads, + final int port, + final ServerSocketConfiguration configuration) { + + if (numThreads <= 0) { + throw new IllegalArgumentException("Number of threads may not be less than or equal to zero."); + } else if (configuration == null) { + throw new IllegalArgumentException("Server socket configuration may not be null."); + } + + this.numThreads = numThreads; + this.port = port; + this.configuration = configuration; + } + + /** + * Implements the action to perform when a new socket request is received. + * This class will close the socket. + * + * @param socket the socket + */ + public abstract void dispatchRequest(final Socket socket); + + public void start() throws IOException { + + if (isRunning()) { + return; + } + + try { + serverSocket = SocketUtils.createServerSocket(port, configuration); + } catch (KeyManagementException | UnrecoverableKeyException | NoSuchAlgorithmException | KeyStoreException | CertificateException e) { + throw new IOException(e); + } + + final ThreadFactory defaultThreadFactory = Executors.defaultThreadFactory(); + executorService = Executors.newFixedThreadPool(numThreads, new ThreadFactory() { + private final AtomicLong threadCounter = new AtomicLong(0L); + + @Override + public Thread newThread(final Runnable r) { + final Thread newThread = defaultThreadFactory.newThread(r); + newThread.setName("Process NCM Request-" + threadCounter.incrementAndGet()); + return newThread; + } + }); + + final ExecutorService runnableExecServiceRef = executorService; + final ServerSocket runnableServerSocketRef = serverSocket; + + final Thread t = new Thread(new Runnable() { + @Override + public void run() { + while (runnableExecServiceRef.isShutdown() == false) { + Socket socket = null; + try { + try { + socket = runnableServerSocketRef.accept(); + if (configuration.getSocketTimeout() != null) { + socket.setSoTimeout(configuration.getSocketTimeout()); + } + } catch (final SocketTimeoutException ste) { + // nobody connected to us. Go ahead and call closeQuietly just to make sure we don't leave + // any sockets lingering + SocketUtils.closeQuietly(socket); + continue; + } catch (final SocketException se) { + logger.warn("Failed to communicate with " + (socket == null ? "Unknown Host" : socket.getInetAddress().getHostName()) + " due to " + se, se); + SocketUtils.closeQuietly(socket); + continue; + } catch (final Throwable t) { + logger.warn("Socket Listener encountered exception: " + t, t); + SocketUtils.closeQuietly(socket); + continue; + } + + final Socket finalSocket = socket; + runnableExecServiceRef.execute(new Runnable() { + @Override + public void run() { + try { + dispatchRequest(finalSocket); + } catch (final Throwable t) { + logger.warn("Dispatching socket request encountered exception due to: " + t, t); + } finally { + SocketUtils.closeQuietly(finalSocket); + } + } + }); + } catch (final Throwable t) { + logger.error("Socket Listener encountered exception: " + t, t); + SocketUtils.closeQuietly(socket); + } + } + } + }); + t.setName("Cluster Socket Listener"); + t.start(); + } + + public boolean isRunning() { + return (executorService != null && executorService.isShutdown() == false); + } + + public void stop() throws IOException { + + if (isRunning() == false) { + return; + } + + // shutdown executor service + try { + if (getShutdownListenerSeconds() <= 0) { + executorService.shutdownNow(); + } else { + executorService.shutdown(); + } + executorService.awaitTermination(getShutdownListenerSeconds(), TimeUnit.SECONDS); + } catch (final InterruptedException ex) { + Thread.currentThread().interrupt(); + } finally { + if (executorService.isTerminated()) { + logger.info("Socket Listener has been terminated successfully."); + } else { + logger.warn("Socket Listener has not terminated properly. There exists an uninterruptable thread that will take an indeterminate amount of time to stop."); + } + } + + // shutdown server socket + SocketUtils.closeQuietly(serverSocket); + + } + + public int getShutdownListenerSeconds() { + return shutdownListenerSeconds.get(); + } + + public void setShutdownListenerSeconds(final int shutdownListenerSeconds) { + this.shutdownListenerSeconds.set(shutdownListenerSeconds); + } + + public ServerSocketConfiguration getConfiguration() { + return configuration; + } + + public int getPort() { + if (isRunning()) { + return serverSocket.getLocalPort(); + } else { + return port; + } + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java new file mode 100644 index 0000000000..fb6a00c6ac --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/SocketUtils.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io.socket; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.ServerSocket; +import java.net.Socket; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLServerSocket; +import javax.net.ssl.SSLSocket; + +import org.apache.nifi.logging.NiFiLog; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author unattributed + */ +public final class SocketUtils { + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketUtils.class)); + + public static Socket createSocket(final InetSocketAddress address, final SocketConfiguration config) throws IOException { + if (address == null) { + throw new IllegalArgumentException("Socket address may not be null."); + } else if (config == null) { + throw new IllegalArgumentException("Configuration may not be null."); + } + + final Socket socket; + + final SSLContext sslContext; + try { + sslContext = config.createSSLContext(); + } catch (final Exception e) { + throw new IOException("Could not create SSLContext", e); + } + + if (sslContext == null) { + socket = new Socket(address.getHostName(), address.getPort()); + } else { + socket = sslContext.getSocketFactory().createSocket(address.getHostName(), address.getPort()); + } + + if (config.getSocketTimeout() != null) { + socket.setSoTimeout(config.getSocketTimeout()); + } + + if (config.getReuseAddress() != null) { + socket.setReuseAddress(config.getReuseAddress()); + } + + if (config.getReceiveBufferSize() != null) { + socket.setReceiveBufferSize(config.getReceiveBufferSize()); + } + + if (config.getSendBufferSize() != null) { + socket.setSendBufferSize(config.getSendBufferSize()); + } + + if (config.getTrafficClass() != null) { + socket.setTrafficClass(config.getTrafficClass()); + } + + if (config.getKeepAlive() != null) { + socket.setKeepAlive(config.getKeepAlive()); + } + + if (config.getOobInline() != null) { + socket.setOOBInline(config.getOobInline()); + } + + if (config.getTcpNoDelay() != null) { + socket.setTcpNoDelay(config.getTcpNoDelay()); + } + + return socket; + } + + public static ServerSocket createServerSocket(final int port, final ServerSocketConfiguration config) throws IOException, KeyManagementException, UnrecoverableKeyException, NoSuchAlgorithmException, KeyStoreException, CertificateException { + if (config == null) { + throw new NullPointerException("Configuration may not be null."); + } + + final SSLContext sslContext = config.createSSLContext(); + final ServerSocket serverSocket; + if (sslContext == null) { + serverSocket = new ServerSocket(port); + } else { + serverSocket = sslContext.getServerSocketFactory().createServerSocket(port); + ((SSLServerSocket) serverSocket).setNeedClientAuth(config.getNeedClientAuth()); + } + + if (config.getSocketTimeout() != null) { + serverSocket.setSoTimeout(config.getSocketTimeout()); + } + + if (config.getReuseAddress() != null) { + serverSocket.setReuseAddress(config.getReuseAddress()); + } + + if (config.getReceiveBufferSize() != null) { + serverSocket.setReceiveBufferSize(config.getReceiveBufferSize()); + } + + return serverSocket; + } + + public static void closeQuietly(final Socket socket) { + if (socket == null) { + return; + } + + try { + try { + // can't shudown input/output individually with secure sockets + if ((socket instanceof SSLSocket) == false) { + if (socket.isInputShutdown() == false) { + socket.shutdownInput(); + } + if (socket.isOutputShutdown() == false) { + socket.shutdownOutput(); + } + } + } finally { + if (socket.isClosed() == false) { + socket.close(); + } + } + } catch (final Exception ex) { + logger.debug("Failed to close socket due to: " + ex, ex); + } + } + + public static void closeQuietly(final ServerSocket serverSocket) { + if (serverSocket == null) { + return; + } + + try { + serverSocket.close(); + } catch (final Exception ex) { + logger.debug("Failed to close server socket due to: " + ex, ex); + } + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java new file mode 100644 index 0000000000..7a62813222 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableService.java @@ -0,0 +1,43 @@ +/* + * 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.io.socket.multicast; + +import java.net.InetSocketAddress; + +/** + * A service that may be discovered at runtime. A service is defined as having a + * unique case-sensitive service name and a socket address where it is + * available. + * + * @author unattributed + */ +public interface DiscoverableService { + + /** + * The service's name. Two services are considered equal if they have the + * same case sensitive service name. + * + * @return the service's name + */ + String getServiceName(); + + /** + * @return the service's address + */ + InetSocketAddress getServiceAddress(); + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java new file mode 100644 index 0000000000..5f378b94ef --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/DiscoverableServiceImpl.java @@ -0,0 +1,78 @@ +/* + * 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.io.socket.multicast; + +import java.net.InetSocketAddress; +import org.apache.commons.lang3.StringUtils; + +/** + * A basic implementation of the DiscoverableService interface. To services are + * considered equal if they have the same case-sensitive service name. + * + * @author unattributed + */ +public class DiscoverableServiceImpl implements DiscoverableService { + + private final String serviceName; + + private final InetSocketAddress serviceAddress; + + public DiscoverableServiceImpl(final String serviceName, final InetSocketAddress serviceAddress) { + if (StringUtils.isBlank(serviceName)) { + throw new IllegalArgumentException("Service name may not be null or empty."); + } else if (serviceAddress == null) { + throw new IllegalArgumentException("Service address may not be null."); + } + this.serviceName = serviceName; + this.serviceAddress = serviceAddress; + } + + @Override + public InetSocketAddress getServiceAddress() { + return serviceAddress; + } + + @Override + public String getServiceName() { + return serviceName; + } + + @Override + public String toString() { + return String.format("[Discoverable Service: %s available at %s:%d]", serviceName, serviceAddress.getHostName(), serviceAddress.getPort()); + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (!(obj instanceof DiscoverableService)) { + return false; + } + final DiscoverableService other = (DiscoverableService) obj; + return !((this.serviceName == null) ? (other.getServiceName() != null) : !this.serviceName.equals(other.getServiceName())); + } + + @Override + public int hashCode() { + int hash = 5; + hash = 53 * hash + (this.serviceName != null ? this.serviceName.hashCode() : 0); + return hash; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java new file mode 100644 index 0000000000..ea0b72a7d8 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastConfiguration.java @@ -0,0 +1,99 @@ +/* + * 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.io.socket.multicast; + +/** + * @author unattributed + */ +public final class MulticastConfiguration { + + private MulticastTimeToLive ttl = DEFAULT_MULTICAST_TTL; + + private Integer socketTimeout; + + private Integer receiveBufferSize; + + private Integer sendBufferSize; + + private Boolean reuseAddress; + + private Integer trafficClass; + + private Boolean loopbackMode; + + public static final MulticastTimeToLive DEFAULT_MULTICAST_TTL = MulticastTimeToLive.SAME_SUBNET; + + public MulticastTimeToLive getTtl() { + return ttl; + } + + public void setTtl(final MulticastTimeToLive ttl) { + if (ttl == null) { + throw new NullPointerException("Multicast TTL may not be null."); + } + this.ttl = ttl; + } + + public Integer getSocketTimeout() { + return socketTimeout; + } + + public void setSocketTimeout(Integer socketTimeout) { + this.socketTimeout = socketTimeout; + } + + public Boolean getReuseAddress() { + return reuseAddress; + } + + public void setReuseAddress(Boolean reuseAddress) { + this.reuseAddress = reuseAddress; + } + + public Integer getReceiveBufferSize() { + return receiveBufferSize; + } + + public void setReceiveBufferSize(Integer receiveBufferSize) { + this.receiveBufferSize = receiveBufferSize; + } + + public Integer getSendBufferSize() { + return sendBufferSize; + } + + public void setSendBufferSize(Integer sendBufferSize) { + this.sendBufferSize = sendBufferSize; + } + + public Integer getTrafficClass() { + return trafficClass; + } + + public void setTrafficClass(Integer trafficClass) { + this.trafficClass = trafficClass; + } + + public Boolean getLoopbackMode() { + return loopbackMode; + } + + public void setLoopbackMode(Boolean loopbackMode) { + this.loopbackMode = loopbackMode; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java new file mode 100644 index 0000000000..e562c25fcd --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastListener.java @@ -0,0 +1,193 @@ +/* + * 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.io.socket.multicast; + +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements a listener for protocol messages sent over multicast. If a message + * is of type MulticastProtocolMessage, then the underlying protocol message is + * passed to the handler. If the receiving handler produces a message response, + * then the message is wrapped with a MulticastProtocolMessage before being sent + * to the originator. + * + * @author unattributed + */ +public abstract class MulticastListener { + + // constants + private static final int DEFAULT_SHUTDOWN_LISTENER_SECONDS = 5; + private static final int DEFAULT_MAX_PACKET_SIZE_BYTES = 512; + + private static final Logger logger = new org.apache.nifi.logging.NiFiLog(LoggerFactory.getLogger(MulticastListener.class)); + + // immutable members + private final int numThreads; + private final InetSocketAddress multicastAddress; + private final MulticastConfiguration configuration; + + private volatile ExecutorService executorService; // volatile to guarantee most current value is visible + private volatile MulticastSocket multicastSocket; // volatile to guarantee most current value is visible + + private int shutdownListenerSeconds = DEFAULT_SHUTDOWN_LISTENER_SECONDS; + private int maxPacketSizeBytes = DEFAULT_MAX_PACKET_SIZE_BYTES; + + public MulticastListener( + final int numThreads, + final InetSocketAddress multicastAddress, + final MulticastConfiguration configuration) { + + if (numThreads <= 0) { + throw new IllegalArgumentException("Number of threads may not be less than or equal to zero."); + } else if (multicastAddress == null) { + throw new IllegalArgumentException("Multicast address may not be null."); + } else if (multicastAddress.getAddress().isMulticastAddress() == false) { + throw new IllegalArgumentException("Multicast group must be a Class D address."); + } else if (configuration == null) { + throw new IllegalArgumentException("Multicast configuration may not be null."); + } + + this.numThreads = numThreads; + this.multicastAddress = multicastAddress; + this.configuration = configuration; + } + + /** + * Implements the action to perform when a new datagram is received. This + * class must not close the multicast socket. + * + * @param multicastSocket + * @param packet the datagram socket + */ + public abstract void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet); + + public void start() throws IOException { + + if (isRunning()) { + return; + } + + multicastSocket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration); + multicastSocket.joinGroup(multicastAddress.getAddress()); + + executorService = Executors.newFixedThreadPool(numThreads); + + final ExecutorService runnableExecServiceRef = executorService; + final MulticastSocket runnableMulticastSocketRef = multicastSocket; + + new Thread(new Runnable() { + @Override + public void run() { + while (runnableExecServiceRef.isShutdown() == false) { + try { + final byte[] buf = new byte[maxPacketSizeBytes]; + final DatagramPacket packet = new DatagramPacket(buf, maxPacketSizeBytes); + runnableMulticastSocketRef.receive(packet); + runnableExecServiceRef.execute(new Runnable() { + @Override + public void run() { + dispatchRequest(multicastSocket, packet); + } + }); + } catch (final SocketException | SocketTimeoutException ste) { + /* ignore so that we can accept connections in approximately a non-blocking fashion */ + } catch (final Exception e) { + logger.warn("Cluster protocol receiver encountered exception: " + e, e); + } + } + } + }).start(); + } + + public boolean isRunning() { + return (executorService != null && executorService.isShutdown() == false); + } + + public void stop() throws IOException { + + if (isRunning() == false) { + return; + } + + // shutdown executor service + try { + if (getShutdownListenerSeconds() <= 0) { + executorService.shutdownNow(); + } else { + executorService.shutdown(); + } + executorService.awaitTermination(getShutdownListenerSeconds(), TimeUnit.SECONDS); + } catch (final InterruptedException ex) { + Thread.currentThread().interrupt(); + } finally { + if (executorService.isTerminated()) { + logger.info("Multicast Listener has been terminated successfully."); + } else { + logger.warn("Multicast Listener has not terminated properly. There exists an uninterruptable thread that will take an indeterminate amount of time to stop."); + } + } + + // shutdown server socket + if (multicastSocket.isClosed() == false) { + multicastSocket.leaveGroup(multicastAddress.getAddress()); + multicastSocket.close(); + } + + } + + public int getShutdownListenerSeconds() { + return shutdownListenerSeconds; + } + + public void setShutdownListenerSeconds(final int shutdownListenerSeconds) { + this.shutdownListenerSeconds = shutdownListenerSeconds; + } + + public int getMaxPacketSizeBytes() { + return maxPacketSizeBytes; + } + + public void setMaxPacketSizeBytes(int maxPacketSizeBytes) { + if (maxPacketSizeBytes <= 0) { + throw new IllegalArgumentException("Max packet size must be greater than zero bytes."); + } + this.maxPacketSizeBytes = maxPacketSizeBytes; + } + + public MulticastConfiguration getConfiguration() { + return configuration; + } + + public InetSocketAddress getMulticastAddress() { + return multicastAddress; + } + + public int getNumThreads() { + return numThreads; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java new file mode 100644 index 0000000000..c254c117e0 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServiceDiscovery.java @@ -0,0 +1,34 @@ +/* + * 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.io.socket.multicast; + +import java.net.InetSocketAddress; + +/** + * Defines the interface for discovering services based on name. Services are + * expected to be exposed via socket address and port. + * + * @author unattributed + */ +public interface MulticastServiceDiscovery extends ServiceDiscovery { + + /** + * @return the multicast address + */ + InetSocketAddress getMulticastAddress(); + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java new file mode 100644 index 0000000000..a3cff9b451 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastServicesBroadcaster.java @@ -0,0 +1,33 @@ +/* + * 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.io.socket.multicast; + +import java.net.InetSocketAddress; + +/** + * Defines the interface for broadcasting a service via multicast. + * + * @author unattributed + */ +public interface MulticastServicesBroadcaster extends ServicesBroadcaster { + + /** + * @return the multicast address + */ + InetSocketAddress getMulticastAddress(); + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java new file mode 100644 index 0000000000..dad117331c --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastTimeToLive.java @@ -0,0 +1,50 @@ +/* + * 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.io.socket.multicast; + +/** + * @author unattributed + */ +public enum MulticastTimeToLive { + + SAME_HOST(0), + SAME_SUBNET(1), + SAME_SITE(32), + SAME_REGION(64), + SAME_CONTINENT(128), + UNRESTRICTED(255); + + private final int ttl; + + MulticastTimeToLive(final int ttl) { + this.ttl = ttl; + } + + public int getTtl() { + return ttl; + } + + public MulticastTimeToLive valueOfByTtl(final int ttl) { + for (final MulticastTimeToLive value : values()) { + if (value.getTtl() == ttl) { + return value; + } + } + return null; + } + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java new file mode 100644 index 0000000000..8a8b7c034c --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/MulticastUtils.java @@ -0,0 +1,109 @@ +/* + * 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.io.socket.multicast; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.MulticastSocket; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author unattributed + */ +public final class MulticastUtils { + + private static final Logger logger = new org.apache.nifi.logging.NiFiLog(LoggerFactory.getLogger(MulticastUtils.class)); + + public static MulticastSocket createMulticastSocket(final MulticastConfiguration config) throws IOException { + return createMulticastSocket(0, config); + } + + public static MulticastSocket createMulticastSocket(final int port, final MulticastConfiguration config) throws IOException { + if (config == null) { + throw new IllegalArgumentException("Configuration may not be null."); + } + + final MulticastSocket socket; + if (port <= 0) { + socket = new MulticastSocket(); + } else { + socket = new MulticastSocket(port); + } + socket.setTimeToLive(config.getTtl().getTtl()); + + if (config.getSocketTimeout() != null) { + socket.setSoTimeout(config.getSocketTimeout()); + } + + if (config.getReuseAddress() != null) { + socket.setReuseAddress(config.getReuseAddress()); + } + + if (config.getReceiveBufferSize() != null) { + socket.setReceiveBufferSize(config.getReceiveBufferSize()); + } + + if (config.getSendBufferSize() != null) { + socket.setSendBufferSize(config.getSendBufferSize()); + } + + if (config.getTrafficClass() != null) { + socket.setTrafficClass(config.getTrafficClass()); + } + + if (config.getLoopbackMode() != null) { + socket.setLoopbackMode(config.getLoopbackMode()); + } + + return socket; + } + + public static void closeQuietly(final MulticastSocket socket) { + + if (socket == null) { + return; + } + + try { + socket.close(); + } catch (final Exception ex) { + logger.debug("Failed to close multicast socket due to: " + ex, ex); + } + + } + + public static void closeQuietly(final MulticastSocket socket, final InetAddress groupAddress) { + + if (socket == null) { + return; + } + + try { + socket.leaveGroup(groupAddress); + } catch (final Exception ex) { + logger.debug("Failed to leave multicast group due to: " + ex, ex); + } + + try { + socket.close(); + } catch (final Exception ex) { + logger.debug("Failed to close multicast socket due to: " + ex, ex); + } + + } +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java new file mode 100644 index 0000000000..173146ecff --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServiceDiscovery.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io.socket.multicast; + +/** + * Defines a generic interface for discovering services. + * + * @author unattributed + */ +public interface ServiceDiscovery { + + /** + * @return the discovered service + */ + DiscoverableService getService(); + +} diff --git a/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java new file mode 100644 index 0000000000..86260d8882 --- /dev/null +++ b/commons/nifi-socket-utils/src/main/java/org/apache/nifi/io/socket/multicast/ServicesBroadcaster.java @@ -0,0 +1,56 @@ +/* + * 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.io.socket.multicast; + +import java.util.Set; + +/** + * Defines the interface for broadcasting a collection of services for client + * discovery. + * + * @author unattributed + */ +public interface ServicesBroadcaster { + + /** + * @return the delay in milliseconds to wait between successive broadcasts + */ + int getBroadcastDelayMs(); + + /** + * @return the broadcasted services + */ + Set getServices(); + + /** + * Adds the given service to the set of broadcasted services. + * + * @param service a service + * @return true if the service was added to the set; false a service with + * the given service name already exists in the set. + */ + boolean addService(DiscoverableService service); + + /** + * Removes the service with the given service name from the set. + * + * @param serviceName a service name + * @return true if the service was removed; false otherwise + */ + boolean removeService(String serviceName); + +} diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java new file mode 100644 index 0000000000..b5240c948b --- /dev/null +++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/ServerMain.java @@ -0,0 +1,141 @@ +/* + * 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.io.nio.example; + +import java.io.IOException; +import java.util.Calendar; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.io.nio.BufferPool; +import org.apache.nifi.io.nio.ChannelListener; +import org.apache.nifi.io.nio.consumer.StreamConsumer; +import org.apache.nifi.io.nio.consumer.StreamConsumerFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * @author none + */ +public final class ServerMain { + + private static final Logger LOGGER = LoggerFactory.getLogger(ServerMain.class); + + public static void main(final String[] args) throws IOException { + System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.io.nio", "debug"); + + final ScheduledExecutorService executor = Executors.newScheduledThreadPool(10); + final Map> consumerMap = new ConcurrentHashMap<>(); + final BufferPool bufferPool = new BufferPool(10, 5 << 20, false, 40.0); + ChannelListener listener = null; + try { + executor.scheduleWithFixedDelay(bufferPool, 0L, 5L, TimeUnit.SECONDS); + listener = new ChannelListener(5, new ExampleStreamConsumerFactory(executor, consumerMap), bufferPool, 5, TimeUnit.MILLISECONDS); + listener.setChannelReaderSchedulingPeriod(50L, TimeUnit.MILLISECONDS); + listener.addDatagramChannel(null, 20000, 32 << 20); + LOGGER.info("Listening for UDP data on port 20000"); + listener.addServerSocket(null, 20001, 64 << 20); + LOGGER.info("listening for TCP connections on port 20001"); + listener.addServerSocket(null, 20002, 64 << 20); + LOGGER.info("listening for TCP connections on port 20002"); + final Calendar endTime = Calendar.getInstance(); + endTime.add(Calendar.MINUTE, 30); + while (true) { + processAllConsumers(consumerMap); + if (endTime.before(Calendar.getInstance())) { + break; // time to shut down + } + } + } finally { + if (listener != null) { + LOGGER.info("Shutting down server...."); + listener.shutdown(1L, TimeUnit.SECONDS); + LOGGER.info("Consumer map size = " + consumerMap.size()); + while (consumerMap.size() > 0) { + processAllConsumers(consumerMap); + } + LOGGER.info("Consumer map size = " + consumerMap.size()); + } + executor.shutdown(); + } + } + + private static void processAllConsumers(final Map> consumerMap) { + final Set deadConsumers = new HashSet<>(); + for (final Map.Entry> entry : consumerMap.entrySet()) { + if (entry.getKey().isConsumerFinished()) { + entry.getValue().cancel(true); + deadConsumers.add(entry.getKey()); + } + } + for (final StreamConsumer consumer : deadConsumers) { + LOGGER.debug("removing consumer " + consumer); + consumerMap.remove(consumer); + } + } + + public static final class ConsumerRunner implements Runnable { + + private final StreamConsumer consumer; + + public ConsumerRunner(final StreamConsumer consumer) { + this.consumer = consumer; + } + + @Override + public void run() { + if (consumer.isConsumerFinished()) { + return; + } + try { + consumer.process(); + } catch (IOException ex) { + LOGGER.error("", ex); + } + } + } + + public static final class ExampleStreamConsumerFactory implements StreamConsumerFactory { + + final ScheduledExecutorService executor; + final Map> consumerMap; + + public ExampleStreamConsumerFactory(final ScheduledExecutorService executor, final Map> consumerMap) { + this.executor = executor; + this.consumerMap = consumerMap; + } + + @Override + public StreamConsumer newInstance(final String streamId) { + final StreamConsumer consumer = new UselessStreamConsumer(streamId); + final ScheduledFuture future = executor.scheduleWithFixedDelay(new ConsumerRunner(consumer), 0L, 10L, TimeUnit.MILLISECONDS); + consumerMap.put(consumer, future); + LOGGER.info("Added consumer: " + consumer); + return consumer; + } + } + +} diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java new file mode 100644 index 0000000000..b3d214e8e7 --- /dev/null +++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/TCPClient.java @@ -0,0 +1,86 @@ +/* + * 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.io.nio.example; + +import java.io.IOException; +import java.net.Socket; +import java.net.SocketException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author none + */ +public class TCPClient { + + private static final Logger logger = LoggerFactory.getLogger(TCPClient.class); + + public static void main(final String[] args) throws Exception { + final byte[] bytes = TCPClient.makeBytes(); + Thread first = new Thread(new Runnable() { + + @Override + public void run() { + try { + for (int i = 0; i < 10; i++) { + sendData(20001, bytes); + } + } catch (Exception e) { + logger.error("Blew exception", e); + } + } + }); + Thread second = new Thread(new Runnable() { + + @Override + public void run() { + try { + for (int i = 0; i < 10; i++) { + sendData(20002, bytes); + } + } catch (Exception e) { + logger.error("Blew exception", e); + } + } + }); + first.start(); + second.start(); + } + + public static byte[] makeBytes() { + byte[] bytes = new byte[2 << 20]; + return bytes; + } + + private static void sendData(final int port, final byte[] bytes) throws SocketException, IOException, InterruptedException { + long totalBytes; + try (Socket sock = new Socket("localhost", port)) { + sock.setTcpNoDelay(true); + sock.setSoTimeout(2000); + totalBytes = 0L; + logger.info("socket established " + sock + " to port " + port + " now waiting 5 seconds to send anything..."); + Thread.sleep(5000L); + for (int i = 0; i < 1000; i++) { + sock.getOutputStream().write(bytes); + totalBytes += bytes.length; + } sock.getOutputStream().flush(); + } + logger.info("Total bytes sent: " + totalBytes + " to port " + port); + } + +} diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java new file mode 100644 index 0000000000..90f4c42015 --- /dev/null +++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UDPClient.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io.nio.example; + +import java.net.DatagramPacket; +import java.net.DatagramSocket; +import java.net.InetSocketAddress; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author none + */ +public class UDPClient { + + private static final Logger LOGGER = LoggerFactory.getLogger(UDPClient.class); + + public static void main(final String[] args) throws Exception { + final byte[] buffer = UDPClient.makeBytes(); + final DatagramPacket packet = new DatagramPacket(buffer, buffer.length, new InetSocketAddress("localhost", 20000)); + final DatagramSocket socket = new DatagramSocket(); + final long startTime = System.nanoTime(); + for (int i = 0; i < 819200; i++) { // 100 MB + socket.send(packet); + } + final long endTime = System.nanoTime(); + final long durationMillis = (endTime - startTime) / 1000000; + LOGGER.info("Sent all UDP packets without any obvious errors | duration ms= " + durationMillis); + } + + public static byte[] makeBytes() { + byte[] bytes = new byte[128]; + return bytes; + } + +} diff --git a/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java new file mode 100644 index 0000000000..9ec26e9401 --- /dev/null +++ b/commons/nifi-socket-utils/src/test/java/org/apache/nifi/io/nio/example/UselessStreamConsumer.java @@ -0,0 +1,43 @@ +/* + * 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.io.nio.example; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import org.apache.nifi.io.nio.consumer.AbstractStreamConsumer; + +/** + * + * @author none + */ +public class UselessStreamConsumer extends AbstractStreamConsumer { + + public UselessStreamConsumer(final String id) { + super(id); + } + + @Override + protected void processBuffer(final ByteBuffer buffer) throws IOException { + } + + @Override + protected void onConsumerDone() { + System.err.println("IN consumer done"); + } + +} diff --git a/commons/nifi-socket-utils/src/test/resources/log4j.xml b/commons/nifi-socket-utils/src/test/resources/log4j.xml new file mode 100644 index 0000000000..8e9376901f --- /dev/null +++ b/commons/nifi-socket-utils/src/test/resources/log4j.xml @@ -0,0 +1,36 @@ + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/commons/nifi-stream-utils/.gitignore b/commons/nifi-stream-utils/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/commons/nifi-stream-utils/.gitignore @@ -0,0 +1 @@ +/target diff --git a/commons/nifi-stream-utils/pom.xml b/commons/nifi-stream-utils/pom.xml new file mode 100644 index 0000000000..04135759b3 --- /dev/null +++ b/commons/nifi-stream-utils/pom.xml @@ -0,0 +1,42 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-stream-utils + 0.0.1-SNAPSHOT + jar + NiFi Stream Utils + + + + + + maven-compiler-plugin + + 1.6 + 1.6 + + + + + + diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java new file mode 100644 index 0000000000..57adb8c726 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedInputStream.java @@ -0,0 +1,37 @@ +/* + * 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.io; + +import java.io.InputStream; + +/** + * This class is a slight modification of the BufferedInputStream in the java.io + * package. The modification is that this implementation does not provide + * synchronization on method calls, which means that this class is not suitable + * for use by multiple threads. However, the absence of these synchronized + * blocks results in potentially much better performance. + */ +public class BufferedInputStream extends java.io.BufferedInputStream { + + public BufferedInputStream(final InputStream in) { + super(in); + } + + public BufferedInputStream(final InputStream in, final int size) { + super(in, size); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java new file mode 100644 index 0000000000..56caf655a8 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/BufferedOutputStream.java @@ -0,0 +1,140 @@ +/* + * 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.io; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +/** + * This class is a slight modification of the + * {@link java.io.BufferedOutputStream} class. This implementation differs in + * that it does not mark methods as synchronized. This means that this class is + * not suitable for writing by multiple concurrent threads. However, the removal + * of the synchronized keyword results in potentially much better performance. + */ +public class BufferedOutputStream extends FilterOutputStream { + + /** + * The internal buffer where data is stored. + */ + protected byte buf[]; + + /** + * The number of valid bytes in the buffer. This value is always in the + * range 0 through buf.length; elements + * buf[0] through buf[count-1] contain valid byte data. + */ + protected int count; + + /** + * Creates a new buffered output stream to write data to the specified + * underlying output stream. + * + * @param out the underlying output stream. + */ + public BufferedOutputStream(OutputStream out) { + this(out, 8192); + } + + /** + * Creates a new buffered output stream to write data to the specified + * underlying output stream with the specified buffer size. + * + * @param out the underlying output stream. + * @param size the buffer size. + * @exception IllegalArgumentException if size <= 0. + */ + public BufferedOutputStream(OutputStream out, int size) { + super(out); + if (size <= 0) { + throw new IllegalArgumentException("Buffer size <= 0"); + } + buf = new byte[size]; + } + + /** + * Flush the internal buffer + */ + private void flushBuffer() throws IOException { + if (count > 0) { + out.write(buf, 0, count); + count = 0; + } + } + + /** + * Writes the specified byte to this buffered output stream. + * + * @param b the byte to be written. + * @exception IOException if an I/O error occurs. + */ + @Override + public void write(int b) throws IOException { + if (count >= buf.length) { + flushBuffer(); + } + buf[count++] = (byte) b; + } + + /** + * Writes len bytes from the specified byte array starting at + * offset off to this buffered output stream. + * + *

+ * Ordinarily this method stores bytes from the given array into this + * stream's buffer, flushing the buffer to the underlying output stream as + * needed. If the requested length is at least as large as this stream's + * buffer, however, then this method will flush the buffer and write the + * bytes directly to the underlying output stream. Thus redundant + * BufferedOutputStreams will not copy data unnecessarily. + * + * @param b the data. + * @param off the start offset in the data. + * @param len the number of bytes to write. + * @exception IOException if an I/O error occurs. + */ + @Override + public void write(byte b[], int off, int len) throws IOException { + if (len >= buf.length) { + /* If the request length exceeds the size of the output buffer, + flush the output buffer and then write the data directly. + In this way buffered streams will cascade harmlessly. */ + flushBuffer(); + out.write(b, off, len); + return; + } + if (len >= buf.length - count) { + flushBuffer(); + } + System.arraycopy(b, off, buf, count, len); + count += len; + } + + /** + * Flushes this buffered output stream. This forces any buffered output + * bytes to be written out to the underlying output stream. + * + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public void flush() throws IOException { + flushBuffer(); + out.flush(); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java new file mode 100644 index 0000000000..9defab4281 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayInputStream.java @@ -0,0 +1,250 @@ +/* + * 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.io; + +import java.io.InputStream; + +/** + * This class performs the same function as java.io.ByteArrayInputStream but + * does not mark its methods as synchronized + */ +public class ByteArrayInputStream extends InputStream { + + /** + * An array of bytes that was provided by the creator of the stream. + * Elements buf[0] through buf[count-1] are the + * only bytes that can ever be read from the stream; element + * buf[pos] is the next byte to be read. + */ + protected byte buf[]; + + /** + * The index of the next character to read from the input stream buffer. + * This value should always be nonnegative and not larger than the value of + * count. The next byte to be read from the input stream buffer + * will be buf[pos]. + */ + protected int pos; + + /** + * The currently marked position in the stream. ByteArrayInputStream objects + * are marked at position zero by default when constructed. They may be + * marked at another position within the buffer by the mark() + * method. The current buffer position is set to this point by the + * reset() method. + *

+ * If no mark has been set, then the value of mark is the offset passed to + * the constructor (or 0 if the offset was not supplied). + * + * @since JDK1.1 + */ + protected int mark = 0; + + /** + * The index one greater than the last valid character in the input stream + * buffer. This value should always be nonnegative and not larger than the + * length of buf. It is one greater than the position of the + * last byte within buf that can ever be read from the input + * stream buffer. + */ + protected int count; + + /** + * Creates a ByteArrayInputStream so that it uses + * buf as its buffer array. The buffer array is not copied. The + * initial value of pos is 0 and the initial value + * of count is the length of buf. + * + * @param buf the input buffer. + */ + public ByteArrayInputStream(byte buf[]) { + this.buf = buf; + this.pos = 0; + this.count = buf.length; + } + + /** + * Creates ByteArrayInputStream that uses buf as + * its buffer array. The initial value of pos is + * offset and the initial value of count is the + * minimum of offset+length and buf.length. The + * buffer array is not copied. The buffer's mark is set to the specified + * offset. + * + * @param buf the input buffer. + * @param offset the offset in the buffer of the first byte to read. + * @param length the maximum number of bytes to read from the buffer. + */ + public ByteArrayInputStream(byte buf[], int offset, int length) { + this.buf = buf; + this.pos = offset; + this.count = Math.min(offset + length, buf.length); + this.mark = offset; + } + + /** + * Reads the next byte of data from this input stream. The value byte is + * returned as an int in the range 0 to + * 255. If no byte is available because the end of the stream + * has been reached, the value -1 is returned. + *

+ * This read method cannot block. + * + * @return the next byte of data, or -1 if the end of the + * stream has been reached. + */ + @Override + public int read() { + return (pos < count) ? (buf[pos++] & 0xff) : -1; + } + + /** + * Reads up to len bytes of data into an array of bytes from + * this input stream. If pos equals count, then + * -1 is returned to indicate end of file. Otherwise, the + * number k of bytes read is equal to the smaller of + * len and count-pos. If k is + * positive, then bytes buf[pos] through + * buf[pos+k-1] are copied into b[off] through + * b[off+k-1] in the manner performed by + * System.arraycopy. The value k is added into + * pos and k is returned. + *

+ * This read method cannot block. + * + * @param b the buffer into which the data is read. + * @param off the start offset in the destination array b + * @param len the maximum number of bytes read. + * @return the total number of bytes read into the buffer, or + * -1 if there is no more data because the end of the stream + * has been reached. + * @exception NullPointerException If b is null. + * @exception IndexOutOfBoundsException If off is negative, + * len is negative, or len is greater than + * b.length - off + */ + @Override + public int read(byte b[], int off, int len) { + if (b == null) { + throw new NullPointerException(); + } else if (off < 0 || len < 0 || len > b.length - off) { + throw new IndexOutOfBoundsException(); + } + + if (pos >= count) { + return -1; + } + + int avail = count - pos; + if (len > avail) { + len = avail; + } + if (len <= 0) { + return 0; + } + System.arraycopy(buf, pos, b, off, len); + pos += len; + return len; + } + + /** + * Skips n bytes of input from this input stream. Fewer bytes + * might be skipped if the end of the input stream is reached. The actual + * number k of bytes to be skipped is equal to the smaller of + * n and count-pos. The value k is + * added into pos and k is returned. + * + * @param n the number of bytes to be skipped. + * @return the actual number of bytes skipped. + */ + @Override + public long skip(long n) { + long k = count - pos; + if (n < k) { + k = n < 0 ? 0 : n; + } + + pos += k; + return k; + } + + /** + * Returns the number of remaining bytes that can be read (or skipped over) + * from this input stream. + *

+ * The value returned is count - pos, which is the number + * of bytes remaining to be read from the input buffer. + * + * @return the number of remaining bytes that can be read (or skipped over) + * from this input stream without blocking. + */ + @Override + public int available() { + return count - pos; + } + + /** + * Tests if this InputStream supports mark/reset. The + * markSupported method of ByteArrayInputStream + * always returns true. + * + * @since JDK1.1 + */ + @Override + public boolean markSupported() { + return true; + } + + /** + * Set the current marked position in the stream. ByteArrayInputStream + * objects are marked at position zero by default when constructed. They may + * be marked at another position within the buffer by this method. + *

+ * If no mark has been set, then the value of the mark is the offset passed + * to the constructor (or 0 if the offset was not supplied). + * + *

+ * Note: The readAheadLimit for this class has no meaning. + * + * @since JDK1.1 + */ + @Override + public void mark(int readAheadLimit) { + mark = pos; + } + + /** + * Resets the buffer to the marked position. The marked position is 0 unless + * another position was marked or an offset was specified in the + * constructor. + */ + @Override + public void reset() { + pos = mark; + } + + /** + * Closing a ByteArrayInputStream has no effect. The methods in + * this class can be called after the stream has been closed without + * generating an IOException. + *

+ */ + @Override + public void close() { + } + +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java new file mode 100644 index 0000000000..b504c235f5 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteArrayOutputStream.java @@ -0,0 +1,250 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.OutputStream; +import java.io.UnsupportedEncodingException; +import java.util.Arrays; + +/** + * This class provides a more efficient implementation of the + * java.io.ByteArrayOutputStream. The efficiency is gained in two ways: + *

    + *
  • The write methods are not synchronized
  • + *
  • The class provides {@link #getUnderlyingBuffer()} and + * {@link #getBufferLength()}, which can be used to access the underlying byte + * array directly, rather than the System.arraycopy that {@link #toByteArray()} + * uses + *
+ * + */ +public class ByteArrayOutputStream extends OutputStream { + + /** + * The buffer where data is stored. + */ + protected byte buf[]; + + /** + * The number of valid bytes in the buffer. + */ + protected int count; + + /** + * Creates a new byte array output stream. The buffer capacity is initially + * 32 bytes, though its size increases if necessary. + */ + public ByteArrayOutputStream() { + this(32); + } + + /** + * Creates a new byte array output stream, with a buffer capacity of the + * specified size, in bytes. + * + * @param size the initial size. + * @exception IllegalArgumentException if size is negative. + */ + public ByteArrayOutputStream(int size) { + if (size < 0) { + throw new IllegalArgumentException("Negative initial size: " + + size); + } + buf = new byte[size]; + } + + /** + * Increases the capacity if necessary to ensure that it can hold at least + * the number of elements specified by the minimum capacity argument. + * + * @param minCapacity the desired minimum capacity + * @throws OutOfMemoryError if {@code minCapacity < 0}. This is interpreted + * as a request for the unsatisfiably large capacity + * {@code (long) Integer.MAX_VALUE + (minCapacity - Integer.MAX_VALUE)}. + */ + private void ensureCapacity(int minCapacity) { + // overflow-conscious code + if (minCapacity - buf.length > 0) { + grow(minCapacity); + } + } + + /** + * Increases the capacity to ensure that it can hold at least the number of + * elements specified by the minimum capacity argument. + * + * @param minCapacity the desired minimum capacity + */ + private void grow(int minCapacity) { + // overflow-conscious code + int oldCapacity = buf.length; + int newCapacity = oldCapacity << 1; + if (newCapacity - minCapacity < 0) { + newCapacity = minCapacity; + } + if (newCapacity < 0) { + if (minCapacity < 0) // overflow + { + throw new OutOfMemoryError(); + } + newCapacity = Integer.MAX_VALUE; + } + buf = Arrays.copyOf(buf, newCapacity); + } + + /** + * Writes the specified byte to this byte array output stream. + * + * @param b the byte to be written. + */ + @Override + public void write(int b) { + ensureCapacity(count + 1); + buf[count] = (byte) b; + count += 1; + } + + /** + * Writes len bytes from the specified byte array starting at + * offset off to this byte array output stream. + * + * @param b the data. + * @param off the start offset in the data. + * @param len the number of bytes to write. + */ + @Override + public void write(byte b[], int off, int len) { + if ((off < 0) || (off > b.length) || (len < 0) + || ((off + len) - b.length > 0)) { + throw new IndexOutOfBoundsException(); + } + ensureCapacity(count + len); + System.arraycopy(b, off, buf, count, len); + count += len; + } + + /** + * Writes the complete contents of this byte array output stream to the + * specified output stream argument, as if by calling the output stream's + * write method using out.write(buf, 0, count). + * + * @param out the output stream to which to write the data. + * @exception IOException if an I/O error occurs. + */ + public void writeTo(OutputStream out) throws IOException { + out.write(buf, 0, count); + } + + /** + * Resets the count field of this byte array output stream to + * zero, so that all currently accumulated output in the output stream is + * discarded. The output stream can be used again, reusing the already + * allocated buffer space. + * + * @see java.io.ByteArrayInputStream#count + */ + public void reset() { + count = 0; + } + + /** + * Creates a newly allocated byte array. Its size is the current size of + * this output stream and the valid contents of the buffer have been copied + * into it. + * + * @return the current contents of this output stream, as a byte array. + * @see java.io.ByteArrayOutputStream#size() + */ + public byte toByteArray () + [] { + return Arrays.copyOf(buf, count); + } + + /** + * Returns the current size of the buffer. + * + * @return the value of the count field, which is the number of + * valid bytes in this output stream. + * @see java.io.ByteArrayOutputStream#count + */ + public int size() { + return count; + } + + /** + * Converts the buffer's contents into a string decoding bytes using the + * platform's default character set. The length of the new String + * is a function of the character set, and hence may not be equal to the + * size of the buffer. + * + *

+ * This method always replaces malformed-input and unmappable-character + * sequences with the default replacement string for the platform's default + * character set. The {@linkplain java.nio.charset.CharsetDecoder} class + * should be used when more control over the decoding process is required. + * + * @return String decoded from the buffer's contents. + * @since JDK1.1 + */ + @Override + public String toString() { + return new String(buf, 0, count); + } + + /** + * Converts the buffer's contents into a string by decoding the bytes using + * the specified {@link java.nio.charset.Charset charsetName}. The length of + * the new String is a function of the charset, and hence may not + * be equal to the length of the byte array. + * + *

+ * This method always replaces malformed-input and unmappable-character + * sequences with this charset's default replacement string. The {@link + * java.nio.charset.CharsetDecoder} class should be used when more control + * over the decoding process is required. + * + * @param charsetName the name of a supported + * {@linkplain java.nio.charset.Charset charset} + * @return String decoded from the buffer's contents. + * @exception UnsupportedEncodingException If the named charset is not + * supported + * @since JDK1.1 + */ + public String toString(String charsetName) throws UnsupportedEncodingException { + return new String(buf, 0, count, charsetName); + } + + /** + * Closing a ByteArrayOutputStream has no effect. The methods in + * this class can be called after the stream has been closed without + * generating an IOException. + *

+ * + */ + @Override + public void close() { + } + + public byte[] getUnderlyingBuffer() { + return buf; + } + + public int getBufferLength() { + return count; + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java new file mode 100644 index 0000000000..792cc32a10 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingInputStream.java @@ -0,0 +1,104 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.InputStream; + +public class ByteCountingInputStream extends InputStream { + + private final InputStream in; + private long bytesRead = 0L; + private long bytesSkipped = 0L; + + private long bytesSinceMark = 0L; + + public ByteCountingInputStream(final InputStream in) { + this.in = in; + } + + @Override + public int read() throws IOException { + final int fromSuper = in.read(); + if (fromSuper >= 0) { + bytesRead++; + bytesSinceMark++; + } + return fromSuper; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + final int fromSuper = in.read(b, off, len); + if (fromSuper >= 0) { + bytesRead += fromSuper; + bytesSinceMark += fromSuper; + } + + return fromSuper; + } + + @Override + public int read(byte[] b) throws IOException { + return read(b, 0, b.length); + } + + @Override + public long skip(final long n) throws IOException { + final long skipped = in.skip(n); + if (skipped >= 0) { + bytesSkipped += skipped; + bytesSinceMark += skipped; + } + return skipped; + } + + public long getBytesRead() { + return bytesRead; + } + + public long getBytesSkipped() { + return bytesSkipped; + } + + public long getBytesConsumed() { + return getBytesRead() + getBytesSkipped(); + } + + @Override + public void mark(final int readlimit) { + in.mark(readlimit); + + bytesSinceMark = 0L; + } + + @Override + public boolean markSupported() { + return in.markSupported(); + } + + @Override + public void reset() throws IOException { + in.reset(); + bytesRead -= bytesSinceMark; + } + + @Override + public void close() throws IOException { + in.close(); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java new file mode 100644 index 0000000000..c7b77ffb59 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ByteCountingOutputStream.java @@ -0,0 +1,63 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.OutputStream; + +public class ByteCountingOutputStream extends OutputStream { + + private final OutputStream out; + private long bytesWritten = 0L; + + public ByteCountingOutputStream(final OutputStream out) { + this.out = out; + } + + @Override + public void write(int b) throws IOException { + out.write(b); + bytesWritten++; + } + + @Override + public void write(byte[] b) throws IOException { + write(b, 0, b.length); + } + + ; + + @Override + public void write(byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + bytesWritten += len; + } + + public long getBytesWritten() { + return bytesWritten; + } + + @Override + public void flush() throws IOException { + out.flush(); + } + + @Override + public void close() throws IOException { + out.close(); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java new file mode 100644 index 0000000000..6af06d3c87 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/DataOutputStream.java @@ -0,0 +1,417 @@ +/* + * 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.io; + +import java.io.DataOutput; +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.io.UTFDataFormatException; + +/** + * This class is different from java.io.DataOutputStream in that it does + * synchronize on its methods. + */ +public class DataOutputStream extends FilterOutputStream implements DataOutput { + + /** + * The number of bytes written to the data output stream so far. If this + * counter overflows, it will be wrapped to Integer.MAX_VALUE. + */ + protected int written; + + /** + * bytearr is initialized on demand by writeUTF + */ + private byte[] bytearr = null; + + /** + * Creates a new data output stream to write data to the specified + * underlying output stream. The counter written is set to + * zero. + * + * @param out the underlying output stream, to be saved for later use. + * @see java.io.FilterOutputStream#out + */ + public DataOutputStream(OutputStream out) { + super(out); + } + + /** + * Increases the written counter by the specified value until it reaches + * Integer.MAX_VALUE. + */ + private void incCount(int value) { + int temp = written + value; + if (temp < 0) { + temp = Integer.MAX_VALUE; + } + written = temp; + } + + /** + * Writes the specified byte (the low eight bits of the argument + * b) to the underlying output stream. If no exception is + * thrown, the counter written is incremented by + * 1. + *

+ * Implements the write method of OutputStream. + * + * @param b the byte to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public void write(int b) throws IOException { + out.write(b); + incCount(1); + } + + /** + * Writes len bytes from the specified byte array starting at + * offset off to the underlying output stream. If no exception + * is thrown, the counter written is incremented by + * len. + * + * @param b the data. + * @param off the start offset in the data. + * @param len the number of bytes to write. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public void write(byte b[], int off, int len) throws IOException { + out.write(b, off, len); + incCount(len); + } + + /** + * Flushes this data output stream. This forces any buffered output bytes to + * be written out to the stream. + *

+ * The flush method of DataOutputStream calls the + * flush method of its underlying output stream. + * + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + * @see java.io.OutputStream#flush() + */ + @Override + public void flush() throws IOException { + out.flush(); + } + + /** + * Writes a boolean to the underlying output stream as a 1-byte + * value. The value true is written out as the value + * (byte)1; the value false is written out as the + * value (byte)0. If no exception is thrown, the counter + * written is incremented by 1. + * + * @param v a boolean value to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeBoolean(boolean v) throws IOException { + out.write(v ? 1 : 0); + incCount(1); + } + + /** + * Writes out a byte to the underlying output stream as a + * 1-byte value. If no exception is thrown, the counter written + * is incremented by 1. + * + * @param v a byte value to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeByte(int v) throws IOException { + out.write(v); + incCount(1); + } + + /** + * Writes a short to the underlying output stream as two bytes, + * high byte first. If no exception is thrown, the counter + * written is incremented by 2. + * + * @param v a short to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeShort(int v) throws IOException { + out.write((v >>> 8) & 0xFF); + out.write((v) & 0xFF); + incCount(2); + } + + /** + * Writes a char to the underlying output stream as a 2-byte + * value, high byte first. If no exception is thrown, the counter + * written is incremented by 2. + * + * @param v a char value to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeChar(int v) throws IOException { + out.write((v >>> 8) & 0xFF); + out.write((v) & 0xFF); + incCount(2); + } + + /** + * Writes an int to the underlying output stream as four bytes, + * high byte first. If no exception is thrown, the counter + * written is incremented by 4. + * + * @param v an int to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeInt(int v) throws IOException { + out.write((v >>> 24) & 0xFF); + out.write((v >>> 16) & 0xFF); + out.write((v >>> 8) & 0xFF); + out.write((v) & 0xFF); + incCount(4); + } + + private final byte writeBuffer[] = new byte[8]; + + /** + * Writes a long to the underlying output stream as eight + * bytes, high byte first. In no exception is thrown, the counter + * written is incremented by 8. + * + * @param v a long to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeLong(long v) throws IOException { + writeBuffer[0] = (byte) (v >>> 56); + writeBuffer[1] = (byte) (v >>> 48); + writeBuffer[2] = (byte) (v >>> 40); + writeBuffer[3] = (byte) (v >>> 32); + writeBuffer[4] = (byte) (v >>> 24); + writeBuffer[5] = (byte) (v >>> 16); + writeBuffer[6] = (byte) (v >>> 8); + writeBuffer[7] = (byte) (v); + out.write(writeBuffer, 0, 8); + incCount(8); + } + + /** + * Converts the float argument to an int using the + * floatToIntBits method in class Float, and then + * writes that int value to the underlying output stream as a + * 4-byte quantity, high byte first. If no exception is thrown, the counter + * written is incremented by 4. + * + * @param v a float value to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + * @see java.lang.Float#floatToIntBits(float) + */ + @Override + public final void writeFloat(float v) throws IOException { + writeInt(Float.floatToIntBits(v)); + } + + /** + * Converts the double argument to a long using the + * doubleToLongBits method in class Double, and + * then writes that long value to the underlying output stream + * as an 8-byte quantity, high byte first. If no exception is thrown, the + * counter written is incremented by 8. + * + * @param v a double value to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + * @see java.lang.Double#doubleToLongBits(double) + */ + @Override + public final void writeDouble(double v) throws IOException { + writeLong(Double.doubleToLongBits(v)); + } + + /** + * Writes out the string to the underlying output stream as a sequence of + * bytes. Each character in the string is written out, in sequence, by + * discarding its high eight bits. If no exception is thrown, the counter + * written is incremented by the length of s. + * + * @param s a string of bytes to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeBytes(String s) throws IOException { + int len = s.length(); + for (int i = 0; i < len; i++) { + out.write((byte) s.charAt(i)); + } + incCount(len); + } + + /** + * Writes a string to the underlying output stream as a sequence of + * characters. Each character is written to the data output stream as if by + * the writeChar method. If no exception is thrown, the counter + * written is incremented by twice the length of + * s. + * + * @param s a String value to be written. + * @exception IOException if an I/O error occurs. + * @see java.io.DataOutputStream#writeChar(int) + * @see java.io.FilterOutputStream#out + */ + @Override + public final void writeChars(String s) throws IOException { + int len = s.length(); + for (int i = 0; i < len; i++) { + int v = s.charAt(i); + out.write((v >>> 8) & 0xFF); + out.write((v) & 0xFF); + } + incCount(len * 2); + } + + /** + * Writes a string to the underlying output stream using + * modified UTF-8 + * encoding in a machine-independent manner. + *

+ * First, two bytes are written to the output stream as if by the + * writeShort method giving the number of bytes to follow. This + * value is the number of bytes actually written out, not the length of the + * string. Following the length, each character of the string is output, in + * sequence, using the modified UTF-8 encoding for the character. If no + * exception is thrown, the counter written is incremented by + * the total number of bytes written to the output stream. This will be at + * least two plus the length of str, and at most two plus + * thrice the length of str. + * + * @param str a string to be written. + * @exception IOException if an I/O error occurs. + */ + @Override + public final void writeUTF(String str) throws IOException { + writeUTF(str, this); + } + + /** + * Writes a string to the specified DataOutput using + * modified UTF-8 + * encoding in a machine-independent manner. + *

+ * First, two bytes are written to out as if by the writeShort + * method giving the number of bytes to follow. This value is the number of + * bytes actually written out, not the length of the string. Following the + * length, each character of the string is output, in sequence, using the + * modified UTF-8 encoding for the character. If no exception is thrown, the + * counter written is incremented by the total number of bytes + * written to the output stream. This will be at least two plus the length + * of str, and at most two plus thrice the length of + * str. + * + * @param str a string to be written. + * @param out destination to write to + * @return The number of bytes written out. + * @exception IOException if an I/O error occurs. + */ + static int writeUTF(String str, DataOutput out) throws IOException { + int strlen = str.length(); + int utflen = 0; + int c, count = 0; + + /* use charAt instead of copying String to char array */ + for (int i = 0; i < strlen; i++) { + c = str.charAt(i); + if ((c >= 0x0001) && (c <= 0x007F)) { + utflen++; + } else if (c > 0x07FF) { + utflen += 3; + } else { + utflen += 2; + } + } + + if (utflen > 65535) { + throw new UTFDataFormatException("encoded string too long: " + utflen + " bytes"); + } + + byte[] bytearr = null; + if (out instanceof DataOutputStream) { + DataOutputStream dos = (DataOutputStream) out; + if (dos.bytearr == null || (dos.bytearr.length < (utflen + 2))) { + dos.bytearr = new byte[(utflen * 2) + 2]; + } + bytearr = dos.bytearr; + } else { + bytearr = new byte[utflen + 2]; + } + + bytearr[count++] = (byte) ((utflen >>> 8) & 0xFF); + bytearr[count++] = (byte) ((utflen) & 0xFF); + + int i = 0; + for (i = 0; i < strlen; i++) { + c = str.charAt(i); + if (!((c >= 0x0001) && (c <= 0x007F))) { + break; + } + bytearr[count++] = (byte) c; + } + + for (; i < strlen; i++) { + c = str.charAt(i); + if ((c >= 0x0001) && (c <= 0x007F)) { + bytearr[count++] = (byte) c; + + } else if (c > 0x07FF) { + bytearr[count++] = (byte) (0xE0 | ((c >> 12) & 0x0F)); + bytearr[count++] = (byte) (0x80 | ((c >> 6) & 0x3F)); + bytearr[count++] = (byte) (0x80 | ((c) & 0x3F)); + } else { + bytearr[count++] = (byte) (0xC0 | ((c >> 6) & 0x1F)); + bytearr[count++] = (byte) (0x80 | ((c) & 0x3F)); + } + } + out.write(bytearr, 0, utflen + 2); + return utflen + 2; + } + + /** + * Returns the current value of the counter written, the number + * of bytes written to this data output stream so far. If the counter + * overflows, it will be wrapped to Integer.MAX_VALUE. + * + * @return the value of the written field. + * @see java.io.DataOutputStream#written + */ + public final int size() { + return written; + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java new file mode 100644 index 0000000000..875b838209 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/GZIPOutputStream.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io; + +import java.io.IOException; +import java.io.OutputStream; + +/** + *

+ * This class extends the {@link java.util.zip.GZIPOutputStream} by allowing the + * constructor to provide a compression level, and uses a default value of 1, + * rather than 5. + *

+ */ +public class GZIPOutputStream extends java.util.zip.GZIPOutputStream { + + public static final int DEFAULT_COMPRESSION_LEVEL = 1; + + public GZIPOutputStream(final OutputStream out) throws IOException { + this(out, DEFAULT_COMPRESSION_LEVEL); + } + + public GZIPOutputStream(final OutputStream out, final int compressionLevel) throws IOException { + super(out); + def.setLevel(compressionLevel); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java new file mode 100644 index 0000000000..0ebe16dd62 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/LeakyBucketStreamThrottler.java @@ -0,0 +1,324 @@ +/* + * 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.io; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +public class LeakyBucketStreamThrottler implements StreamThrottler { + + private final int maxBytesPerSecond; + private final BlockingQueue requestQueue = new LinkedBlockingQueue(); + private final ScheduledExecutorService executorService; + private final AtomicBoolean shutdown = new AtomicBoolean(false); + + public LeakyBucketStreamThrottler(final int maxBytesPerSecond) { + this.maxBytesPerSecond = maxBytesPerSecond; + + executorService = Executors.newSingleThreadScheduledExecutor(); + final Runnable task = new Drain(); + executorService.scheduleAtFixedRate(task, 0, 1000, TimeUnit.MILLISECONDS); + } + + @Override + public void close() { + this.shutdown.set(true); + + executorService.shutdown(); + try { + // Should not take more than 2 seconds because we run every second. If it takes more than + // 2 seconds, it is because the Runnable thread is blocking on a write; in this case, + // we will just ignore it and return + executorService.awaitTermination(2, TimeUnit.SECONDS); + } catch (InterruptedException e) { + } + } + + @Override + public OutputStream newThrottledOutputStream(final OutputStream toWrap) { + return new OutputStream() { + @Override + public void write(final int b) throws IOException { + write(new byte[]{(byte) b}, 0, 1); + } + + @Override + public void write(byte[] b) throws IOException { + write(b, 0, b.length); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + final InputStream in = new ByteArrayInputStream(b, off, len); + LeakyBucketStreamThrottler.this.copy(in, toWrap); + } + + @Override + public void close() throws IOException { + toWrap.close(); + } + + @Override + public void flush() throws IOException { + toWrap.flush(); + } + }; + } + + @Override + public InputStream newThrottledInputStream(final InputStream toWrap) { + return new InputStream() { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + @Override + public int read() throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(1); + LeakyBucketStreamThrottler.this.copy(toWrap, baos, 1L); + if (baos.getBufferLength() < 1) { + return -1; + } + + return baos.getUnderlyingBuffer()[0] & 0xFF; + } + + @Override + public int read(final byte[] b) throws IOException { + if(b.length == 0){ + return 0; + } + return read(b, 0, b.length); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + baos.reset(); + final int copied = (int) LeakyBucketStreamThrottler.this.copy(toWrap, baos, len); + if (copied == 0) { + return -1; + } + System.arraycopy(baos.getUnderlyingBuffer(), 0, b, off, copied); + return copied; + } + + @Override + public void close() throws IOException { + toWrap.close(); + } + + @Override + public int available() throws IOException { + return toWrap.available(); + } + }; + } + + @Override + public long copy(final InputStream in, final OutputStream out) throws IOException { + return copy(in, out, -1); + } + + @Override + public long copy(final InputStream in, final OutputStream out, final long maxBytes) throws IOException { + long totalBytesCopied = 0; + boolean finished = false; + while (!finished) { + final long requestMax = (maxBytes < 0) ? Long.MAX_VALUE : maxBytes - totalBytesCopied; + final Request request = new Request(in, out, requestMax); + boolean transferred = false; + while (!transferred) { + if (shutdown.get()) { + throw new IOException("Throttler shutdown"); + } + + try { + transferred = requestQueue.offer(request, 1000, TimeUnit.MILLISECONDS); + } catch (final InterruptedException e) { + throw new IOException("Interrupted", e); + } + } + + final BlockingQueue responseQueue = request.getResponseQueue(); + Response response = null; + while (response == null) { + try { + if (shutdown.get()) { + throw new IOException("Throttler shutdown"); + } + response = responseQueue.poll(1000L, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + throw new IOException("Interrupted", e); + } + } + + if (!response.isSuccess()) { + throw response.getError(); + } + + totalBytesCopied += response.getBytesCopied(); + finished = (response.getBytesCopied() == 0) || (totalBytesCopied >= maxBytes && maxBytes > 0); + } + + return totalBytesCopied; + } + + /** + * This class is responsible for draining water from the leaky bucket. I.e., + * it actually moves the data + */ + private class Drain implements Runnable { + + private final byte[] buffer; + + public Drain() { + final int bufferSize = Math.min(4096, maxBytesPerSecond); + buffer = new byte[bufferSize]; + } + + @Override + public void run() { + final long start = System.currentTimeMillis(); + + int bytesTransferred = 0; + while (bytesTransferred < maxBytesPerSecond) { + final long maxMillisToWait = 1000 - (System.currentTimeMillis() - start); + if (maxMillisToWait < 1) { + return; + } + + try { + final Request request = requestQueue.poll(maxMillisToWait, TimeUnit.MILLISECONDS); + if (request == null) { + return; + } + + final BlockingQueue responseQueue = request.getResponseQueue(); + + final OutputStream out = request.getOutputStream(); + final InputStream in = request.getInputStream(); + + try { + final long requestMax = request.getMaxBytesToCopy(); + long maxBytesToTransfer; + if (requestMax < 0) { + maxBytesToTransfer = Math.min(buffer.length, maxBytesPerSecond - bytesTransferred); + } else { + maxBytesToTransfer = Math.min(requestMax, + Math.min(buffer.length, maxBytesPerSecond - bytesTransferred)); + } + maxBytesToTransfer = Math.max(1L, maxBytesToTransfer); + + final int bytesCopied = fillBuffer(in, maxBytesToTransfer); + out.write(buffer, 0, bytesCopied); + + final Response response = new Response(true, bytesCopied); + responseQueue.put(response); + bytesTransferred += bytesCopied; + } catch (final IOException e) { + final Response response = new Response(e); + responseQueue.put(response); + } + } catch (InterruptedException e) { + } + } + } + + private int fillBuffer(final InputStream in, final long maxBytes) throws IOException { + int bytesRead = 0; + int len; + while (bytesRead < maxBytes && (len = in.read(buffer, bytesRead, (int) Math.min(maxBytes - bytesRead, buffer.length - bytesRead))) > 0) { + bytesRead += len; + } + + return bytesRead; + } + } + + private static class Response { + + private final boolean success; + private final IOException error; + private final int bytesCopied; + + public Response(final boolean success, final int bytesCopied) { + this.success = success; + this.bytesCopied = bytesCopied; + this.error = null; + } + + public Response(final IOException error) { + this.success = false; + this.error = error; + this.bytesCopied = -1; + } + + public boolean isSuccess() { + return success; + } + + public IOException getError() { + return error; + } + + public int getBytesCopied() { + return bytesCopied; + } + } + + private static class Request { + + private final OutputStream out; + private final InputStream in; + private final long maxBytesToCopy; + private final BlockingQueue responseQueue; + + public Request(final InputStream in, final OutputStream out, final long maxBytesToCopy) { + this.out = out; + this.in = in; + this.maxBytesToCopy = maxBytesToCopy; + this.responseQueue = new LinkedBlockingQueue(1); + } + + public BlockingQueue getResponseQueue() { + return this.responseQueue; + } + + public OutputStream getOutputStream() { + return out; + } + + public InputStream getInputStream() { + return in; + } + + public long getMaxBytesToCopy() { + return maxBytesToCopy; + } + + @Override + public String toString() { + return "Request[maxBytes=" + maxBytesToCopy + "]"; + } + } + +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java new file mode 100644 index 0000000000..1fbb093fdd --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableInputStream.java @@ -0,0 +1,56 @@ +/* + * 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.io; + +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; + +/** + * Wraps and InputStream so that the underlying InputStream cannot be closed. + * This is used so that the InputStream can be wrapped with yet another + * InputStream and prevent the outer layer from closing the inner InputStream + */ +public class NonCloseableInputStream extends FilterInputStream { + + private final InputStream toWrap; + + public NonCloseableInputStream(final InputStream toWrap) { + super(toWrap); + this.toWrap = toWrap; + } + + @Override + public int read() throws IOException { + return toWrap.read(); + } + + @Override + public int read(byte[] b) throws IOException { + return toWrap.read(b); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + return toWrap.read(b, off, len); + } + + @Override + public void close() throws IOException { + // do nothing + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java new file mode 100644 index 0000000000..731e409aea --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NonCloseableOutputStream.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io; + +import java.io.FilterOutputStream; +import java.io.IOException; +import java.io.OutputStream; + +public class NonCloseableOutputStream extends FilterOutputStream { + + private final OutputStream out; + + public NonCloseableOutputStream(final OutputStream out) { + super(out); + this.out = out; + } + + @Override + public void write(byte[] b) throws IOException { + out.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + out.write(b, off, len); + } + + @Override + public void write(int b) throws IOException { + out.write(b); + } + + @Override + public void close() throws IOException { + out.flush(); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.java new file mode 100644 index 0000000000..60475d457e --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/NullOutputStream.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.io; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * OutputStream that throws away all data, just like as if writing to /dev/null + */ +public class NullOutputStream extends OutputStream { + + @Override + public void write(final int b) throws IOException { + } + + @Override + public void write(final byte[] b) throws IOException { + } + + @Override + public void write(final byte[] b, int off, int len) throws IOException { + } + + @Override + public void close() throws IOException { + } + + @Override + public void flush() throws IOException { + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java new file mode 100644 index 0000000000..8c2aa80047 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamThrottler.java @@ -0,0 +1,33 @@ +/* + * 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.io; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +public interface StreamThrottler extends Closeable { + + long copy(InputStream in, OutputStream out) throws IOException; + + long copy(InputStream in, OutputStream out, long maxBytes) throws IOException; + + InputStream newThrottledInputStream(final InputStream toWrap); + + OutputStream newThrottledOutputStream(final OutputStream toWrap); +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java new file mode 100644 index 0000000000..1596014291 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/StreamUtils.java @@ -0,0 +1,257 @@ +/* + * 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.io; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.io.exception.BytePatternNotFoundException; +import org.apache.nifi.io.util.NonThreadSafeCircularBuffer; + +public class StreamUtils { + + public static long copy(final InputStream source, final OutputStream destination) throws IOException { + final byte[] buffer = new byte[8192]; + int len; + long totalCount = 0L; + while ((len = source.read(buffer)) > 0) { + destination.write(buffer, 0, len); + totalCount += len; + } + return totalCount; + } + + /** + * Copies numBytes from source to + * destination. If numBytes are not available from + * source, throws EOFException + * + * @param source + * @param destination + * @param numBytes + * @throws IOException + */ + public static void copy(final InputStream source, final OutputStream destination, final long numBytes) throws IOException { + final byte[] buffer = new byte[8192]; + int len; + long bytesLeft = numBytes; + while ((len = source.read(buffer, 0, (int) Math.min(bytesLeft, buffer.length))) > 0) { + destination.write(buffer, 0, len); + bytesLeft -= len; + } + + if (bytesLeft > 0) { + throw new EOFException("Attempted to copy " + numBytes + " bytes but only " + (numBytes - bytesLeft) + " bytes were available"); + } + } + + /** + * Reads data from the given input stream, copying it to the destination + * byte array. If the InputStream has less data than the given byte array, + * throws an EOFException + * + * @param source + * @param destination + * @throws IOException + */ + public static void fillBuffer(final InputStream source, final byte[] destination) throws IOException { + fillBuffer(source, destination, true); + } + + /** + * Reads data from the given input stream, copying it to the destination + * byte array. If the InputStream has less data than the given byte array, + * throws an EOFException if ensureCapacity is true and + * otherwise returns the number of bytes copied + * + * @param source + * @param destination + * @param ensureCapacity whether or not to enforce that the InputStream have + * at least as much data as the capacity of the destination byte array + * @return + * @throws IOException + */ + public static int fillBuffer(final InputStream source, final byte[] destination, final boolean ensureCapacity) throws IOException { + int bytesRead = 0; + int len; + while (bytesRead < destination.length) { + len = source.read(destination, bytesRead, destination.length - bytesRead); + if (len < 0) { + if (ensureCapacity) { + throw new EOFException(); + } else { + break; + } + } + + bytesRead += len; + } + + return bytesRead; + } + + /** + * Copies data from in to out until either we are out of data (returns null) + * or we hit one of the byte patterns identified by the + * stoppers parameter (returns the byte pattern matched). The + * bytes in the stopper will be copied. + * + * @param in + * @param out + * @param maxBytes + * @param stoppers + * @return the byte array matched, or null if end of stream was reached + * @throws IOException + */ + public static byte[] copyInclusive(final InputStream in, final OutputStream out, final int maxBytes, final byte[]... stoppers) throws IOException { + if (stoppers.length == 0) { + return null; + } + + final List circularBuffers = new ArrayList(); + for (final byte[] stopper : stoppers) { + circularBuffers.add(new NonThreadSafeCircularBuffer(stopper)); + } + + long bytesRead = 0; + while (true) { + final int next = in.read(); + if (next == -1) { + return null; + } else if (maxBytes > 0 && ++bytesRead >= maxBytes) { + throw new BytePatternNotFoundException("Did not encounter any byte pattern that was expected; data does not appear to be in the expected format"); + } + + out.write(next); + + for (final NonThreadSafeCircularBuffer circ : circularBuffers) { + if (circ.addAndCompare((byte) next)) { + return circ.getByteArray(); + } + } + } + } + + /** + * Copies data from in to out until either we are out of data (returns null) + * or we hit one of the byte patterns identified by the + * stoppers parameter (returns the byte pattern matched). The + * byte pattern matched will NOT be copied to the output and will be un-read + * from the input. + * + * @param in + * @param out + * @param maxBytes + * @param stoppers + * @return the byte array matched, or null if end of stream was reached + * @throws IOException + */ + public static byte[] copyExclusive(final InputStream in, final OutputStream out, final int maxBytes, final byte[]... stoppers) throws IOException { + if (stoppers.length == 0) { + return null; + } + + int longest = 0; + NonThreadSafeCircularBuffer longestBuffer = null; + final List circularBuffers = new ArrayList(); + for (final byte[] stopper : stoppers) { + final NonThreadSafeCircularBuffer circularBuffer = new NonThreadSafeCircularBuffer(stopper); + if (stopper.length > longest) { + longest = stopper.length; + longestBuffer = circularBuffer; + circularBuffers.add(0, circularBuffer); + } else { + circularBuffers.add(circularBuffer); + } + } + + long bytesRead = 0; + while (true) { + final int next = in.read(); + if (next == -1) { + return null; + } else if (maxBytes > 0 && bytesRead++ > maxBytes) { + throw new BytePatternNotFoundException("Did not encounter any byte pattern that was expected; data does not appear to be in the expected format"); + } + + for (final NonThreadSafeCircularBuffer circ : circularBuffers) { + if (circ.addAndCompare((byte) next)) { + // The longest buffer has some data that may not have been written out yet; we need to make sure + // that we copy out those bytes. + final int bytesToCopy = longest - circ.getByteArray().length; + for (int i = 0; i < bytesToCopy; i++) { + final int oldestByte = longestBuffer.getOldestByte(); + if (oldestByte != -1) { + out.write(oldestByte); + longestBuffer.addAndCompare((byte) 0); + } + } + + return circ.getByteArray(); + } + } + + if (longestBuffer.isFilled()) { + out.write(longestBuffer.getOldestByte()); + } + } + } + + /** + * Skips the specified number of bytes from the InputStream + * + * If unable to skip that number of bytes, throws EOFException + * + * @param stream + * @param bytesToSkip + * @throws IOException + */ + public static void skip(final InputStream stream, final long bytesToSkip) throws IOException { + if (bytesToSkip <= 0) { + return; + } + long totalSkipped = 0L; + + // If we have a FileInputStream, calling skip(1000000) will return 1000000 even if the file is only + // 3 bytes. As a result, we will skip 1 less than the number requested, and then read the last + // byte in order to make sure that we've consumed the number of bytes requested. We then check that + // the final byte, which we read, is not -1. + final long actualBytesToSkip = bytesToSkip - 1; + while (totalSkipped < actualBytesToSkip) { + final long skippedThisIteration = stream.skip(actualBytesToSkip - totalSkipped); + if (skippedThisIteration == 0) { + final int nextByte = stream.read(); + if (nextByte == -1) { + throw new EOFException(); + } else { + totalSkipped++; + } + } + + totalSkipped += skippedThisIteration; + } + + final int lastByte = stream.read(); + if (lastByte == -1) { + throw new EOFException(); + } + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java new file mode 100644 index 0000000000..f2857207bb --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/ZipOutputStream.java @@ -0,0 +1,38 @@ +/* + * 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.io; + +import java.io.OutputStream; + +/** + * This class extends the {@link java.util.zip.ZipOutputStream} by providing a + * constructor that allows the user to specify the compression level. The + * default compression level is 1, as opposed to Java's default of 5. + */ +public class ZipOutputStream extends java.util.zip.ZipOutputStream { + + public static final int DEFAULT_COMPRESSION_LEVEL = 1; + + public ZipOutputStream(final OutputStream out) { + this(out, DEFAULT_COMPRESSION_LEVEL); + } + + public ZipOutputStream(final OutputStream out, final int compressionLevel) { + super(out); + def.setLevel(compressionLevel); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java new file mode 100644 index 0000000000..8935767d17 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/exception/BytePatternNotFoundException.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io.exception; + +import java.io.IOException; + +public class BytePatternNotFoundException extends IOException { + + private static final long serialVersionUID = -4128911284318513973L; + + public BytePatternNotFoundException(final String explanation) { + super(explanation); + } +} diff --git a/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java new file mode 100644 index 0000000000..1b874889e6 --- /dev/null +++ b/commons/nifi-stream-utils/src/main/java/org/apache/nifi/io/util/NonThreadSafeCircularBuffer.java @@ -0,0 +1,69 @@ +/* + * 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.io.util; + +import java.util.Arrays; + +public class NonThreadSafeCircularBuffer { + + private final byte[] lookingFor; + private final int[] buffer; + private int insertionPointer = 0; + private int bufferSize = 0; + + public NonThreadSafeCircularBuffer(final byte[] lookingFor) { + this.lookingFor = lookingFor; + buffer = new int[lookingFor.length]; + Arrays.fill(buffer, -1); + } + + public byte[] getByteArray() { + return lookingFor; + } + + /** + * Returns the oldest byte in the buffer + * + * @return + */ + public int getOldestByte() { + return buffer[insertionPointer]; + } + + public boolean isFilled() { + return bufferSize >= buffer.length; + } + + public boolean addAndCompare(final byte data) { + buffer[insertionPointer] = data; + insertionPointer = (insertionPointer + 1) % lookingFor.length; + + bufferSize++; + if (bufferSize < lookingFor.length) { + return false; + } + + for (int i = 0; i < lookingFor.length; i++) { + final byte compare = (byte) buffer[(insertionPointer + i) % lookingFor.length]; + if (compare != lookingFor[i]) { + return false; + } + } + + return true; + } +} diff --git a/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java b/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java new file mode 100644 index 0000000000..12e1801b25 --- /dev/null +++ b/commons/nifi-stream-utils/src/test/java/org/apache/nifi/io/TestLeakyBucketThrottler.java @@ -0,0 +1,144 @@ +/* + * 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.io; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; + +import org.junit.Ignore; +import org.junit.Test; + +@Ignore("Tests are time-based") +public class TestLeakyBucketThrottler { + + @Test(timeout = 10000) + public void testOutputStreamInterface() throws IOException { + // throttle rate at 1 MB/sec + final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024); + + final byte[] data = new byte[1024 * 1024 * 4]; + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final OutputStream throttledOut = throttler.newThrottledOutputStream(baos); + + final long start = System.currentTimeMillis(); + throttledOut.write(data); + throttler.close(); + final long millis = System.currentTimeMillis() - start; + // should take 4 sec give or take + assertTrue(millis > 3000); + assertTrue(millis < 6000); + } + + @Test(timeout = 10000) + public void testInputStreamInterface() throws IOException { + // throttle rate at 1 MB/sec + final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024); + + final byte[] data = new byte[1024 * 1024 * 4]; + final ByteArrayInputStream bais = new ByteArrayInputStream(data); + final InputStream throttledIn = throttler.newThrottledInputStream(bais); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final byte[] buffer = new byte[4096]; + final long start = System.currentTimeMillis(); + int len; + while ((len = throttledIn.read(buffer)) > 0) { + baos.write(buffer, 0, len); + } + throttler.close(); + final long millis = System.currentTimeMillis() - start; + // should take 4 sec give or take + assertTrue(millis > 3000); + assertTrue(millis < 6000); + baos.close(); + } + + @Test(timeout = 10000) + public void testDirectInterface() throws IOException, InterruptedException { + // throttle rate at 1 MB/sec + final LeakyBucketStreamThrottler throttler = new LeakyBucketStreamThrottler(1024 * 1024); + + // create 3 threads, each sending ~2 MB + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final List threads = new ArrayList(); + for (int i = 0; i < 3; i++) { + final Thread t = new WriterThread(i, throttler, baos); + threads.add(t); + } + + final long start = System.currentTimeMillis(); + for (final Thread t : threads) { + t.start(); + } + + for (final Thread t : threads) { + t.join(); + } + final long elapsed = System.currentTimeMillis() - start; + + throttler.close(); + + // To send 15 MB, it should have taken at least 5 seconds and no more than 7 seconds, to + // allow for busy-ness and the fact that we could write a tiny bit more than the limit. + assertTrue(elapsed > 5000); + assertTrue(elapsed < 7000); + + // ensure bytes were copied out appropriately + assertEquals(3 * (2 * 1024 * 1024 + 1), baos.getBufferLength()); + assertEquals((byte) 'A', baos.getUnderlyingBuffer()[baos.getBufferLength() - 1]); + } + + private static class WriterThread extends Thread { + + private final int idx; + private final byte[] data = new byte[1024 * 1024 * 2 + 1]; + private final LeakyBucketStreamThrottler throttler; + private final OutputStream out; + + public WriterThread(final int idx, final LeakyBucketStreamThrottler throttler, final OutputStream out) { + this.idx = idx; + this.throttler = throttler; + this.out = out; + this.data[this.data.length - 1] = (byte) 'A'; + } + + @Override + public void run() { + long startMillis = System.currentTimeMillis(); + long bytesWritten = 0L; + try { + throttler.copy(new ByteArrayInputStream(data), out); + } catch (IOException e) { + e.printStackTrace(); + return; + } + long now = System.currentTimeMillis(); + long millisElapsed = now - startMillis; + bytesWritten += data.length; + float bytesPerSec = (float) bytesWritten / (float) millisElapsed * 1000F; + System.out.println(idx + " : copied data at a rate of " + bytesPerSec + " bytes/sec"); + } + } + +} diff --git a/commons/nifi-stream-utils/src/test/resources/logback-test.xml b/commons/nifi-stream-utils/src/test/resources/logback-test.xml new file mode 100644 index 0000000000..0f3f60c309 --- /dev/null +++ b/commons/nifi-stream-utils/src/test/resources/logback-test.xml @@ -0,0 +1,48 @@ + + + + + + + %-4r [%t] %-5p %c - %m%n + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/commons/nifi-utils/.gitignore b/commons/nifi-utils/.gitignore new file mode 100755 index 0000000000..12c5231300 --- /dev/null +++ b/commons/nifi-utils/.gitignore @@ -0,0 +1,8 @@ +/target +/target +/target +/target +/target +/target +/target +/target diff --git a/commons/nifi-utils/pom.xml b/commons/nifi-utils/pom.xml new file mode 100644 index 0000000000..8aeccd7b60 --- /dev/null +++ b/commons/nifi-utils/pom.xml @@ -0,0 +1,28 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-utils + 0.0.1-SNAPSHOT + jar + NiFi Utils + diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java new file mode 100644 index 0000000000..e22032bf6e --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/CompoundUpdateMonitor.java @@ -0,0 +1,115 @@ +/* + * 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.io; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.List; + +/** + * An {@link UpdateMonitor} that combines multiple UpdateMonitors + * such that it will indicate a change in a file only if ALL sub-monitors + * indicate a change. The sub-monitors will be applied in the order given and if + * any indicates that the state has not changed, the subsequent sub-monitors may + * not be given a chance to run + */ +public class CompoundUpdateMonitor implements UpdateMonitor { + + private final List monitors; + + public CompoundUpdateMonitor(final UpdateMonitor first, final UpdateMonitor... others) { + monitors = new ArrayList<>(); + monitors.add(first); + for (final UpdateMonitor monitor : others) { + monitors.add(monitor); + } + } + + @Override + public Object getCurrentState(final Path path) throws IOException { + return new DeferredMonitorAction(monitors, path); + } + + private static class DeferredMonitorAction { + + private static final Object NON_COMPUTED_VALUE = new Object(); + + private final List monitors; + private final Path path; + + private final Object[] preCalculated; + + public DeferredMonitorAction(final List monitors, final Path path) { + this.monitors = monitors; + this.path = path; + preCalculated = new Object[monitors.size()]; + + for (int i = 0; i < preCalculated.length; i++) { + preCalculated[i] = NON_COMPUTED_VALUE; + } + } + + private Object getCalculatedValue(final int i) throws IOException { + if (preCalculated[i] == NON_COMPUTED_VALUE) { + preCalculated[i] = monitors.get(i).getCurrentState(path); + } + + return preCalculated[i]; + } + + @Override + public boolean equals(final Object obj) { + // must return true unless ALL DeferredMonitorAction's indicate that they are different + if (obj == null) { + return false; + } + + if (!(obj instanceof DeferredMonitorAction)) { + return false; + } + + final DeferredMonitorAction other = (DeferredMonitorAction) obj; + try { + // Go through each UpdateMonitor's value and check if the value has changed. + for (int i = 0; i < preCalculated.length; i++) { + final Object mine = getCalculatedValue(i); + final Object theirs = other.getCalculatedValue(i); + + if (mine == theirs) { + // same + return true; + } + + if (mine == null && theirs == null) { + // same + return true; + } + + if (mine.equals(theirs)) { + return true; + } + } + } catch (final IOException e) { + return false; + } + + // No DeferredMonitorAction was the same as last time. Therefore, it's not equal + return false; + } + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java new file mode 100644 index 0000000000..f4464652e9 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/LastModifiedMonitor.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.io; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; + +public class LastModifiedMonitor implements UpdateMonitor { + + @Override + public Object getCurrentState(final Path path) throws IOException { + return Files.getLastModifiedTime(path); + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java new file mode 100644 index 0000000000..1326c2aad5 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/MD5SumMonitor.java @@ -0,0 +1,49 @@ +/* + * 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.io; + +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; + +public class MD5SumMonitor implements UpdateMonitor { + + @Override + public Object getCurrentState(final Path path) throws IOException { + final MessageDigest digest; + try { + digest = MessageDigest.getInstance("MD5"); + } catch (final NoSuchAlgorithmException nsae) { + throw new AssertionError(nsae); + } + + try (final FileInputStream fis = new FileInputStream(path.toFile())) { + int len; + final byte[] buffer = new byte[8192]; + while ((len = fis.read(buffer)) > 0) { + digest.update(buffer, 0, len); + } + } + + // Return a ByteBuffer instead of byte[] because we want equals() to do a deep equality + return ByteBuffer.wrap(digest.digest()); + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java new file mode 100644 index 0000000000..785f1ace27 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/SynchronousFileWatcher.java @@ -0,0 +1,123 @@ +/* + * 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.io; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Allows the user to configure a {@link java.nio.file.Path Path} to watch for + * modifications and periodically poll to check if the file has been modified + */ +public class SynchronousFileWatcher { + + private final Path path; + private final long checkUpdateMillis; + private final UpdateMonitor monitor; + private final AtomicReference lastState; + private final Lock resourceLock = new ReentrantLock(); + + public SynchronousFileWatcher(final Path path, final UpdateMonitor monitor) { + this(path, monitor, 0L); + } + + public SynchronousFileWatcher(final Path path, final UpdateMonitor monitor, final long checkMillis) { + if (checkMillis < 0) { + throw new IllegalArgumentException(); + } + + this.path = path; + checkUpdateMillis = checkMillis; + this.monitor = monitor; + + Object currentState; + try { + currentState = monitor.getCurrentState(path); + } catch (final IOException e) { + currentState = null; + } + + this.lastState = new AtomicReference<>(new StateWrapper(currentState)); + } + + /** + * Checks if the file has been updated according to the configured + * {@link UpdateMonitor} and resets the state + * + * @return + * @throws IOException + */ + public boolean checkAndReset() throws IOException { + if (checkUpdateMillis <= 0) { // if checkUpdateMillis <= 0, always check + return checkForUpdate(); + } else { + final StateWrapper stateWrapper = lastState.get(); + if (stateWrapper.getTimestamp() < System.currentTimeMillis() - checkUpdateMillis) { + return checkForUpdate(); + } + return false; + } + } + + private boolean checkForUpdate() throws IOException { + if (resourceLock.tryLock()) { + try { + final StateWrapper wrapper = lastState.get(); + final Object newState = monitor.getCurrentState(path); + if (newState == null && wrapper.getState() == null) { + return false; + } + if (newState == null || wrapper.getState() == null) { + lastState.set(new StateWrapper(newState)); + return true; + } + + final boolean unmodified = newState.equals(wrapper.getState()); + if (!unmodified) { + lastState.set(new StateWrapper(newState)); + } + return !unmodified; + } finally { + resourceLock.unlock(); + } + } else { + return false; + } + } + + private static class StateWrapper { + + private final Object state; + private final long timestamp; + + public StateWrapper(final Object state) { + this.state = state; + this.timestamp = System.currentTimeMillis(); + } + + public Object getState() { + return state; + } + + public long getTimestamp() { + return timestamp; + } + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java b/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java new file mode 100644 index 0000000000..33fb444242 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/io/UpdateMonitor.java @@ -0,0 +1,25 @@ +/* + * 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.io; + +import java.io.IOException; +import java.nio.file.Path; + +public interface UpdateMonitor { + + Object getCurrentState(Path path) throws IOException; +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java new file mode 100644 index 0000000000..92061e0875 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/BooleanHolder.java @@ -0,0 +1,25 @@ +/* + * 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.util; + +public class BooleanHolder extends ObjectHolder { + + public BooleanHolder(final boolean initialValue) { + super(initialValue); + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java new file mode 100644 index 0000000000..9954bfb09f --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java @@ -0,0 +1,204 @@ +/* + * 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.util; + +import java.text.NumberFormat; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class FormatUtils { + + private static final String UNION = "|"; + + // for Data Sizes + private static final double BYTES_IN_KILOBYTE = 1024; + private static final double BYTES_IN_MEGABYTE = BYTES_IN_KILOBYTE * 1024; + private static final double BYTES_IN_GIGABYTE = BYTES_IN_MEGABYTE * 1024; + private static final double BYTES_IN_TERABYTE = BYTES_IN_GIGABYTE * 1024; + + // for Time Durations + private static final String NANOS = join(UNION, "ns", "nano", "nanos", "nanoseconds"); + private static final String MILLIS = join(UNION, "ms", "milli", "millis", "milliseconds"); + private static final String SECS = join(UNION, "s", "sec", "secs", "second", "seconds"); + private static final String MINS = join(UNION, "m", "min", "mins", "minute", "minutes"); + private static final String HOURS = join(UNION, "h", "hr", "hrs", "hour", "hours"); + private static final String DAYS = join(UNION, "d", "day", "days"); + + private static final String VALID_TIME_UNITS = join(UNION, NANOS, MILLIS, SECS, MINS, HOURS, DAYS); + public static final String TIME_DURATION_REGEX = "(\\d+)\\s*(" + VALID_TIME_UNITS + ")"; + public static final Pattern TIME_DURATION_PATTERN = Pattern.compile(TIME_DURATION_REGEX); + + /** + * Formats the specified count by adding commas. + * + * @param count + * @return + */ + public static String formatCount(final long count) { + return NumberFormat.getIntegerInstance().format(count); + } + + /** + * Formats the specified duration in 'mm:ss.SSS' format. + * + * @param sourceDuration + * @param sourceUnit + * @return + */ + public static String formatMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) { + final long millis = TimeUnit.MILLISECONDS.convert(sourceDuration, sourceUnit); + final SimpleDateFormat formatter = new SimpleDateFormat("mm:ss.SSS"); + return formatter.format(new Date(millis)); + } + + /** + * Formats the specified duration in 'HH:mm:ss.SSS' format. + * + * @param sourceDuration + * @param sourceUnit + * @return + */ + public static String formatHoursMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) { + final long millis = TimeUnit.MILLISECONDS.convert(sourceDuration, sourceUnit); + final long millisInHour = TimeUnit.MILLISECONDS.convert(1, TimeUnit.HOURS); + final int hours = (int) (millis / millisInHour); + final long whatsLeft = millis - hours * millisInHour; + + return pad(hours) + ":" + new SimpleDateFormat("mm:ss.SSS").format(new Date(whatsLeft)); + } + + private static String pad(final int val) { + return (val < 10) ? "0" + val : String.valueOf(val); + } + + /** + * Formats the specified data size in human readable format. + * + * @param dataSize Data size in bytes + * @return Human readable format + */ + public static String formatDataSize(final double dataSize) { + // initialize the formatter + final NumberFormat format = NumberFormat.getNumberInstance(); + format.setMaximumFractionDigits(2); + + // check terabytes + double dataSizeToFormat = dataSize / BYTES_IN_TERABYTE; + if (dataSizeToFormat > 1) { + return format.format(dataSizeToFormat) + " TB"; + } + + // check gigabytes + dataSizeToFormat = dataSize / BYTES_IN_GIGABYTE; + if (dataSizeToFormat > 1) { + return format.format(dataSizeToFormat) + " GB"; + } + + // check megabytes + dataSizeToFormat = dataSize / BYTES_IN_MEGABYTE; + if (dataSizeToFormat > 1) { + return format.format(dataSizeToFormat) + " MB"; + } + + // check kilobytes + dataSizeToFormat = dataSize / BYTES_IN_KILOBYTE; + if (dataSizeToFormat > 1) { + return format.format(dataSizeToFormat) + " KB"; + } + + // default to bytes + return format.format(dataSize) + " bytes"; + } + + public static long getTimeDuration(final String value, final TimeUnit desiredUnit) { + final Matcher matcher = TIME_DURATION_PATTERN.matcher(value.toLowerCase()); + if (!matcher.matches()) { + throw new IllegalArgumentException("Value '" + value + "' is not a valid Time Duration"); + } + + final String duration = matcher.group(1); + final String units = matcher.group(2); + TimeUnit specifiedTimeUnit = null; + switch (units.toLowerCase()) { + case "ns": + case "nano": + case "nanos": + case "nanoseconds": + specifiedTimeUnit = TimeUnit.NANOSECONDS; + break; + case "ms": + case "milli": + case "millis": + case "milliseconds": + specifiedTimeUnit = TimeUnit.MILLISECONDS; + break; + case "s": + case "sec": + case "secs": + case "second": + case "seconds": + specifiedTimeUnit = TimeUnit.SECONDS; + break; + case "m": + case "min": + case "mins": + case "minute": + case "minutes": + specifiedTimeUnit = TimeUnit.MINUTES; + break; + case "h": + case "hr": + case "hrs": + case "hour": + case "hours": + specifiedTimeUnit = TimeUnit.HOURS; + break; + case "d": + case "day": + case "days": + specifiedTimeUnit = TimeUnit.DAYS; + break; + } + + final long durationVal = Long.parseLong(duration); + return desiredUnit.convert(durationVal, specifiedTimeUnit); + } + + public static String formatUtilization(final double utilization) { + return utilization + "%"; + } + + private static String join(final String delimiter, final String... values) { + if (values.length == 0) { + return ""; + } else if (values.length == 1) { + return values[0]; + } + + final StringBuilder sb = new StringBuilder(); + sb.append(values[0]); + for (int i = 1; i < values.length; i++) { + sb.append(delimiter).append(values[i]); + } + + return sb.toString(); + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java new file mode 100644 index 0000000000..213bbc0443 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/IntegerHolder.java @@ -0,0 +1,54 @@ +/* + * 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.util; + +public class IntegerHolder extends ObjectHolder { + + public IntegerHolder(final int initialValue) { + super(initialValue); + } + + public int addAndGet(final int delta) { + final int curValue = get(); + final int newValue = curValue + delta; + set(newValue); + return newValue; + } + + public int getAndAdd(final int delta) { + final int curValue = get(); + final int newValue = curValue + delta; + set(newValue); + return curValue; + } + + public int incrementAndGet() { + return addAndGet(1); + } + + public int getAndIncrement() { + return getAndAdd(1); + } + + public int decrementAndGet() { + return addAndGet(-1); + } + + public int getAndDecrement() { + return getAndAdd(-1); + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java new file mode 100644 index 0000000000..ef70ce805a --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/LongHolder.java @@ -0,0 +1,60 @@ +/* + * 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.util; + +/** + * Wraps a Long value so that it can be declared final and still be + * accessed from which inner classes; the functionality is similar to that of an + * AtomicLong, but operations on this class are not atomic. This results in + * greater performance when the atomicity is not needed. + */ +public class LongHolder extends ObjectHolder { + + public LongHolder(final long initialValue) { + super(initialValue); + } + + public long addAndGet(final long delta) { + final long curValue = get(); + final long newValue = curValue + delta; + set(newValue); + return newValue; + } + + public long getAndAdd(final long delta) { + final long curValue = get(); + final long newValue = curValue + delta; + set(newValue); + return curValue; + } + + public long incrementAndGet() { + return addAndGet(1); + } + + public long getAndIncrement() { + return getAndAdd(1); + } + + public long decrementAndGet() { + return addAndGet(-1L); + } + + public long getAndDecrement() { + return getAndAdd(-1L); + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java new file mode 100644 index 0000000000..a58ec6a10e --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/ObjectHolder.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util; + +/** + * A bean that holds a single value of type T. + * + * @param + */ +public class ObjectHolder { + + private T value; + + public ObjectHolder(final T initialValue) { + this.value = initialValue; + } + + public T get() { + return value; + } + + public void set(T value) { + this.value = value; + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java new file mode 100644 index 0000000000..c0bb830770 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/RingBuffer.java @@ -0,0 +1,292 @@ +/* + * 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.util; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +/** + * Thread-safe implementation of a RingBuffer + * + * @param + */ +public class RingBuffer { + + private final Object[] buffer; + private int insertionPointer = 0; + private boolean filled = false; + + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + private final Lock writeLock = rwLock.writeLock(); + + public RingBuffer(final int size) { + buffer = new Object[size]; + } + + /** + * Adds the given value to the RingBuffer and returns the value that was + * removed in order to make room. + * + * @param value + * @return + */ + @SuppressWarnings("unchecked") + public T add(final T value) { + Objects.requireNonNull(value); + + writeLock.lock(); + try { + final Object removed = buffer[insertionPointer]; + + buffer[insertionPointer] = value; + + if (insertionPointer == buffer.length - 1) { + filled = true; + } + + insertionPointer = (insertionPointer + 1) % buffer.length; + return (T) removed; + } finally { + writeLock.unlock(); + } + } + + public int getSize() { + readLock.lock(); + try { + return filled ? buffer.length : insertionPointer; + } finally { + readLock.unlock(); + } + } + + public List getSelectedElements(final Filter filter) { + return getSelectedElements(filter, Integer.MAX_VALUE); + } + + public List getSelectedElements(final Filter filter, final int maxElements) { + final List selected = new ArrayList<>(1000); + int numSelected = 0; + readLock.lock(); + try { + for (int i = 0; i < buffer.length && numSelected < maxElements; i++) { + final int idx = (insertionPointer + i) % buffer.length; + final Object val = buffer[idx]; + if (val == null) { + continue; + } + + @SuppressWarnings("unchecked") + final T element = (T) val; + if (filter.select(element)) { + selected.add(element); + numSelected++; + } + } + } finally { + readLock.unlock(); + } + return selected; + } + + public int countSelectedElements(final Filter filter) { + int numSelected = 0; + readLock.lock(); + try { + for (int i = 0; i < buffer.length; i++) { + final int idx = (insertionPointer + i) % buffer.length; + final Object val = buffer[idx]; + if (val == null) { + continue; + } + + @SuppressWarnings("unchecked") + final T element = (T) val; + if (filter.select(element)) { + numSelected++; + } + } + } finally { + readLock.unlock(); + } + + return numSelected; + } + + /** + * Removes all elements from the RingBuffer that match the given filter + * + * @param filter + * @return + */ + public int removeSelectedElements(final Filter filter) { + int count = 0; + + writeLock.lock(); + try { + for (int i = 0; i < buffer.length; i++) { + final int idx = (insertionPointer + i + 1) % buffer.length; + final Object val = buffer[idx]; + if (val == null) { + continue; + } + + @SuppressWarnings("unchecked") + final T element = (T) val; + + if (filter.select(element)) { + buffer[idx] = null; + } + } + } finally { + writeLock.unlock(); + } + + return count; + } + + public List asList() { + return getSelectedElements(new Filter() { + @Override + public boolean select(final T value) { + return true; + } + }); + } + + public T getOldestElement() { + readLock.lock(); + try { + return getElementData(insertionPointer); + } finally { + readLock.unlock(); + } + } + + public T getNewestElement() { + readLock.lock(); + try { + int index = (insertionPointer == 0) ? buffer.length : insertionPointer - 1; + return getElementData(index); + } finally { + readLock.unlock(); + } + } + + @SuppressWarnings("unchecked") + private T getElementData(final int index) { + readLock.lock(); + try { + return (T) buffer[index]; + } finally { + readLock.unlock(); + } + } + + /** + * Iterates over each element in the RingBuffer, calling the + * {@link ForEachEvaluator#evaluate(Object) evaluate} method on each element + * in the RingBuffer. If the Evaluator returns {@code false}, the method + * will skip all remaining elements in the RingBuffer; otherwise, the next + * element will be evaluated until all elements have been evaluated. + * + * @param evaluator + */ + public void forEach(final ForEachEvaluator evaluator) { + forEach(evaluator, IterationDirection.FORWARD); + } + + /** + * Iterates over each element in the RingBuffer, calling the + * {@link ForEachEvaluator#evaluate(Object) evaluate} method on each element + * in the RingBuffer. If the Evaluator returns {@code false}, the method + * will skip all remaining elements in the RingBuffer; otherwise, the next + * element will be evaluated until all elements have been evaluated. + * + * @param evaluator + * @param iterationDirection the order in which to iterate over the elements + * in the RingBuffer + */ + public void forEach(final ForEachEvaluator evaluator, final IterationDirection iterationDirection) { + readLock.lock(); + try { + final int startIndex; + final int endIndex; + final int increment; + + if (iterationDirection == IterationDirection.FORWARD) { + startIndex = 0; + endIndex = buffer.length - 1; + increment = 1; + } else { + startIndex = buffer.length - 1; + endIndex = 0; + increment = -1; + } + + for (int i = startIndex; (iterationDirection == IterationDirection.FORWARD ? i <= endIndex : i >= endIndex); i += increment) { + final int idx = (insertionPointer + i) % buffer.length; + final Object val = buffer[idx]; + if (val == null) { + continue; + } + + @SuppressWarnings("unchecked") + final T element = (T) val; + if (!evaluator.evaluate(element)) { + return; + } + } + } finally { + readLock.unlock(); + } + } + + public static interface Filter { + + boolean select(S value); + } + + /** + * Defines an interface that can be used to iterate over all of the elements + * in the RingBuffer via the {@link #forEach} method + * + * @param + */ + public static interface ForEachEvaluator { + + /** + * Evaluates the given element and returns {@code true} if the next + * element should be evaluated, {@code false} otherwise + * + * @param value + * @return + */ + boolean evaluate(S value); + } + + public static enum IterationDirection { + + FORWARD, + BACKWARD; + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java new file mode 100644 index 0000000000..cd119305f3 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/StopWatch.java @@ -0,0 +1,127 @@ +/* + * 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.util; + +import java.util.concurrent.TimeUnit; + +public final class StopWatch { + + private long startNanos = -1L; + private long duration = -1L; + + /** + * Creates a StopWatch but does not start it + */ + public StopWatch() { + this(false); + } + + /** + * @param autoStart whether or not the timer should be started automatically + */ + public StopWatch(final boolean autoStart) { + if (autoStart) { + start(); + } + } + + public void start() { + this.startNanos = System.nanoTime(); + this.duration = -1L; + } + + public void stop() { + if (startNanos < 0) { + throw new IllegalStateException("StopWatch has not been started"); + } + this.duration = System.nanoTime() - startNanos; + this.startNanos = -1L; + } + + /** + * Returns the amount of time that the StopWatch was running. + * + * @param timeUnit + * @return + * + * @throws IllegalStateException if the StopWatch has not been stopped via + * {@link #stop()} + */ + public long getDuration(final TimeUnit timeUnit) { + if (duration < 0) { + throw new IllegalStateException("Cannot get duration until StopWatch has been stopped"); + } + return timeUnit.convert(duration, TimeUnit.NANOSECONDS); + } + + /** + * Returns the amount of time that has elapsed since the timer was started. + * + * @param timeUnit + * @return + */ + public long getElapsed(final TimeUnit timeUnit) { + return timeUnit.convert(System.nanoTime() - startNanos, TimeUnit.NANOSECONDS); + } + + public String calculateDataRate(final long bytes) { + final double seconds = (double) duration / 1000000000.0D; + final long dataSize = (long) (bytes / seconds); + return FormatUtils.formatDataSize(dataSize) + "/sec"; + } + + public String getDuration() { + final StringBuilder sb = new StringBuilder(); + + long duration = this.duration; + final long minutes = (duration > 60000000000L) ? (duration / 60000000000L) : 0L; + duration -= TimeUnit.NANOSECONDS.convert(minutes, TimeUnit.MINUTES); + + final long seconds = (duration > 1000000000L) ? (duration / 1000000000L) : 0L; + duration -= TimeUnit.NANOSECONDS.convert(seconds, TimeUnit.SECONDS); + + final long millis = (duration > 1000000L) ? (duration / 1000000L) : 0L; + duration -= TimeUnit.NANOSECONDS.convert(millis, TimeUnit.MILLISECONDS); + + final long nanos = duration % 1000000L; + + if (minutes > 0) { + sb.append(minutes).append(" minutes"); + } + + if (seconds > 0) { + if (minutes > 0) { + sb.append(", "); + } + + sb.append(seconds).append(" seconds"); + } + + if (millis > 0) { + if (seconds > 0) { + sb.append(", "); + } + + sb.append(millis).append(" millis"); + } + if (seconds == 0 && millis == 0) { + sb.append(nanos).append(" nanos"); + } + + return sb.toString(); + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java new file mode 100644 index 0000000000..63736ed2f6 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/Tuple.java @@ -0,0 +1,83 @@ +/* + * 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.util; + +/** + * + * @author unattrib + * @param + * @param + */ +public class Tuple { + + final A key; + final B value; + + public Tuple(A key, B value) { + this.key = key; + this.value = value; + } + + public A getKey() { + return key; + } + + public B getValue() { + return value; + } + + @Override + public boolean equals(final Object other) { + if (other == null) { + return false; + } + if (other == this) { + return true; + } + if (!(other instanceof Tuple)) { + return false; + } + + final Tuple tuple = (Tuple) other; + if (key == null) { + if (tuple.key != null) { + return false; + } + } else { + if (!key.equals(tuple.key)) { + return false; + } + } + + if (value == null) { + if (tuple.value != null) { + return false; + } + } else { + if (!value.equals(tuple.value)) { + return false; + } + } + + return true; + } + + @Override + public int hashCode() { + return 581 + (this.key == null ? 0 : this.key.hashCode()) + (this.value == null ? 0 : this.value.hashCode()); + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java new file mode 100644 index 0000000000..a8d7e82650 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugDisabledTimedLock.java @@ -0,0 +1,67 @@ +/* + * 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.util.concurrency; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; + +public class DebugDisabledTimedLock implements DebuggableTimedLock { + + private final Lock lock; + + public DebugDisabledTimedLock(final Lock lock) { + this.lock = lock; + } + + /** + * + * @return + */ + @Override + public boolean tryLock() { + return lock.tryLock(); + } + + /** + * + * @param timeout + * @param timeUnit + * @return + */ + @Override + public boolean tryLock(final long timeout, final TimeUnit timeUnit) { + try { + return lock.tryLock(timeout, timeUnit); + } catch (InterruptedException e) { + return false; + } + } + + /** + * + */ + @Override + public void lock() { + lock.lock(); + } + + @Override + public void unlock(final String task) { + lock.unlock(); + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java new file mode 100644 index 0000000000..f082168582 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebugEnabledTimedLock.java @@ -0,0 +1,136 @@ +/* + * 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.util.concurrency; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DebugEnabledTimedLock implements DebuggableTimedLock { + + private final Lock lock; + private final Logger logger; + private long lockTime = 0L; + + private final Map lockIterations = new HashMap<>(); + private final Map lockNanos = new HashMap<>(); + + private final String name; + private final int iterationFrequency; + + public DebugEnabledTimedLock(final Lock lock, final String name, final int iterationFrequency) { + this.lock = lock; + this.name = name; + this.iterationFrequency = iterationFrequency; + logger = LoggerFactory.getLogger(TimedLock.class.getName() + "." + name); + } + + /** + * + * @return + */ + @Override + public boolean tryLock() { + logger.trace("Trying to obtain Lock: {}", name); + final boolean success = lock.tryLock(); + if (!success) { + logger.trace("TryLock failed for Lock: {}", name); + return false; + } + logger.trace("TryLock successful"); + + return true; + } + + /** + * + * @param timeout + * @param timeUnit + * @return + */ + @Override + public boolean tryLock(final long timeout, final TimeUnit timeUnit) { + logger.trace("Trying to obtain Lock {} with a timeout of {} {}", name, timeout, timeUnit); + final boolean success; + try { + success = lock.tryLock(timeout, timeUnit); + } catch (final InterruptedException ie) { + return false; + } + + if (!success) { + logger.trace("TryLock failed for Lock {} with a timeout of {} {}", name, timeout, timeUnit); + return false; + } + logger.trace("TryLock successful"); + return true; + } + + /** + * + */ + @Override + public void lock() { + logger.trace("Obtaining Lock {}", name); + lock.lock(); + lockTime = System.nanoTime(); + logger.trace("Obtained Lock {}", name); + } + + /** + * + * @param task + */ + @Override + public void unlock(final String task) { + if (lockTime <= 0L) { + lock.unlock(); + return; + } + + logger.trace("Releasing Lock {}", name); + final long nanosLocked = System.nanoTime() - lockTime; + + Long startIterations = lockIterations.get(task); + if (startIterations == null) { + startIterations = 0L; + } + final long iterations = startIterations + 1L; + lockIterations.put(task, iterations); + + Long startNanos = lockNanos.get(task); + if (startNanos == null) { + startNanos = 0L; + } + final long totalNanos = startNanos + nanosLocked; + lockNanos.put(task, totalNanos); + + lockTime = -1L; + + lock.unlock(); + logger.trace("Released Lock {}", name); + + if (iterations % iterationFrequency == 0) { + logger.debug("Lock {} held for {} nanos for task: {}; total lock iterations: {}; total lock nanos: {}", name, nanosLocked, task, iterations, totalNanos); + } + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java new file mode 100644 index 0000000000..69da6e8cfe --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/DebuggableTimedLock.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util.concurrency; + +import java.util.concurrent.TimeUnit; + +public interface DebuggableTimedLock { + + void lock(); + + boolean tryLock(long timePeriod, TimeUnit timeUnit); + + boolean tryLock(); + + void unlock(String task); +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java new file mode 100644 index 0000000000..532d3c3d04 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/concurrency/TimedLock.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util.concurrency; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TimedLock { + + private final DebugEnabledTimedLock enabled; + private final DebugDisabledTimedLock disabled; + + private final Logger logger; + + public TimedLock(final Lock lock, final String name, final int iterationFrequency) { + this.enabled = new DebugEnabledTimedLock(lock, name, iterationFrequency); + this.disabled = new DebugDisabledTimedLock(lock); + + logger = LoggerFactory.getLogger(TimedLock.class.getName() + "." + name); + } + + private DebuggableTimedLock getLock() { + return logger.isDebugEnabled() ? enabled : disabled; + } + + public boolean tryLock() { + return getLock().tryLock(); + } + + public boolean tryLock(final long timeout, final TimeUnit timeUnit) { + return getLock().tryLock(timeout, timeUnit); + } + + public void lock() { + getLock().lock(); + } + + public void unlock(final String task) { + getLock().unlock(task); + } + +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.java new file mode 100644 index 0000000000..2b958971fc --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/EntityAccess.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.util.timebuffer; + +public interface EntityAccess { + + T aggregate(T oldValue, T toAdd); + + T createNew(); + + long getTimestamp(T entity); +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java new file mode 100644 index 0000000000..193abc60e2 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/LongEntityAccess.java @@ -0,0 +1,43 @@ +/* + * 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.util.timebuffer; + +public class LongEntityAccess implements EntityAccess { + + @Override + public TimestampedLong aggregate(TimestampedLong oldValue, TimestampedLong toAdd) { + if (oldValue == null && toAdd == null) { + return new TimestampedLong(0L); + } else if (oldValue == null) { + return toAdd; + } else if (toAdd == null) { + return oldValue; + } + + return new TimestampedLong(oldValue.getValue() + toAdd.getValue()); + } + + @Override + public TimestampedLong createNew() { + return new TimestampedLong(0L); + } + + @Override + public long getTimestamp(TimestampedLong entity) { + return entity == null ? 0L : entity.getTimestamp(); + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java new file mode 100644 index 0000000000..dd8e5232c4 --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimedBuffer.java @@ -0,0 +1,114 @@ +/* + * 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.util.timebuffer; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +public class TimedBuffer { + + private final int numBins; + private final EntitySum[] bins; + private final EntityAccess entityAccess; + private final TimeUnit binPrecision; + + @SuppressWarnings("unchecked") + public TimedBuffer(final TimeUnit binPrecision, final int numBins, final EntityAccess accessor) { + this.binPrecision = binPrecision; + this.numBins = numBins + 1; + this.bins = new EntitySum[this.numBins]; + for (int i = 0; i < this.numBins; i++) { + this.bins[i] = new EntitySum<>(binPrecision, numBins, accessor); + } + this.entityAccess = accessor; + } + + public T add(final T entity) { + final int binIdx = (int) (binPrecision.convert(System.currentTimeMillis(), TimeUnit.MILLISECONDS) % numBins); + final EntitySum sum = bins[binIdx]; + + return sum.addOrReset(entity); + } + + public T getAggregateValue(final long sinceEpochMillis) { + final int startBinIdx = (int) (binPrecision.convert(sinceEpochMillis, TimeUnit.MILLISECONDS) % numBins); + + T total = null; + for (int i = 0; i < numBins; i++) { + int binIdx = (startBinIdx + i) % numBins; + final EntitySum bin = bins[binIdx]; + + if (!bin.isExpired()) { + total = entityAccess.aggregate(total, bin.getValue()); + } + } + + return total; + } + + private static class EntitySum { + + private final EntityAccess entityAccess; + private final AtomicReference ref = new AtomicReference<>(); + private final TimeUnit binPrecision; + private final int numConfiguredBins; + + public EntitySum(final TimeUnit binPrecision, final int numConfiguredBins, final EntityAccess aggregator) { + this.binPrecision = binPrecision; + this.entityAccess = aggregator; + this.numConfiguredBins = numConfiguredBins; + } + + private S add(final S event) { + S newValue; + S value; + do { + value = ref.get(); + newValue = entityAccess.aggregate(value, event); + } while (!ref.compareAndSet(value, newValue)); + + return newValue; + } + + public S getValue() { + return ref.get(); + } + + public boolean isExpired() { + // entityAccess.getTimestamp(curValue) represents the time at which the current value + // was last updated. If the last value is less than current time - 1 binPrecision, then it + // means that we've rolled over and need to reset the value. + final long maxExpectedTimePeriod = System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(numConfiguredBins, binPrecision); + + final S curValue = ref.get(); + return (entityAccess.getTimestamp(curValue) < maxExpectedTimePeriod); + } + + public S addOrReset(final S event) { + // entityAccess.getTimestamp(curValue) represents the time at which the current value + // was last updated. If the last value is less than current time - 1 binPrecision, then it + // means that we've rolled over and need to reset the value. + final long maxExpectedTimePeriod = System.currentTimeMillis() - TimeUnit.MILLISECONDS.convert(1, binPrecision); + + final S curValue = ref.get(); + if (entityAccess.getTimestamp(curValue) < maxExpectedTimePeriod) { + ref.compareAndSet(curValue, entityAccess.createNew()); + } + return add(event); + } + } +} diff --git a/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java new file mode 100644 index 0000000000..07d31ea5aa --- /dev/null +++ b/commons/nifi-utils/src/main/java/org/apache/nifi/util/timebuffer/TimestampedLong.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util.timebuffer; + +public class TimestampedLong { + + private final Long value; + private final long timestamp = System.currentTimeMillis(); + + public TimestampedLong(final Long value) { + this.value = value; + } + + public Long getValue() { + return value; + } + + public long getTimestamp() { + return timestamp; + } +} diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.java new file mode 100644 index 0000000000..c796a96935 --- /dev/null +++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestCompoundUpdateMonitor.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.util.timebuffer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotSame; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Path; +import java.util.UUID; + +import org.apache.nifi.io.CompoundUpdateMonitor; +import org.apache.nifi.io.LastModifiedMonitor; +import org.apache.nifi.io.MD5SumMonitor; +import org.apache.nifi.io.UpdateMonitor; + +import org.junit.Test; + +public class TestCompoundUpdateMonitor { + + @Test + public void test() throws IOException { + final UpdateMonitor lastModified = new LastModifiedMonitor(); + final MD5SumMonitor md5 = new MD5SumMonitor(); + final CompoundUpdateMonitor compound = new CompoundUpdateMonitor(lastModified, md5); + + final File file = new File("target/" + UUID.randomUUID().toString()); + if (file.exists()) { + assertTrue(file.delete()); + } + assertTrue(file.createNewFile()); + + final Path path = file.toPath(); + + final Object curState = compound.getCurrentState(path); + final Object state2 = compound.getCurrentState(path); + + assertEquals(curState, state2); + file.setLastModified(System.currentTimeMillis() + 1000L); + final Object state3 = compound.getCurrentState(path); + assertEquals(state2, state3); + + final Object state4 = compound.getCurrentState(path); + assertEquals(state3, state4); + + final long lastModifiedDate = file.lastModified(); + try (final OutputStream out = new FileOutputStream(file)) { + out.write("Hello".getBytes("UTF-8")); + } + + file.setLastModified(lastModifiedDate); + + final Object state5 = compound.getCurrentState(path); + assertNotSame(state4, state5); + } + +} diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java new file mode 100644 index 0000000000..fafffdd24b --- /dev/null +++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestRingBuffer.java @@ -0,0 +1,182 @@ +/* + * 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.util.timebuffer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.nifi.util.RingBuffer; +import org.apache.nifi.util.RingBuffer.ForEachEvaluator; +import org.apache.nifi.util.RingBuffer.IterationDirection; + +import org.junit.Test; + +/** + * + */ +public class TestRingBuffer { + + @Test + public void testAsList() { + final RingBuffer ringBuffer = new RingBuffer<>(10); + + final List emptyList = ringBuffer.asList(); + assertTrue(emptyList.isEmpty()); + + for (int i = 0; i < 3; i++) { + ringBuffer.add(i); + } + + List list = ringBuffer.asList(); + assertEquals(3, list.size()); + for (int i = 0; i < 3; i++) { + assertEquals(Integer.valueOf(i), list.get(i)); + } + + for (int i = 3; i < 10; i++) { + ringBuffer.add(i); + } + + list = ringBuffer.asList(); + assertEquals(10, list.size()); + for (int i = 0; i < 10; i++) { + assertEquals(Integer.valueOf(i), list.get(i)); + } + } + + @Test + public void testIterateForwards() { + final RingBuffer ringBuffer = new RingBuffer<>(10); + + final int[] values = new int[]{3, 5, 20, 7}; + for (final int v : values) { + ringBuffer.add(v); + } + + final AtomicInteger countHolder = new AtomicInteger(0); + ringBuffer.forEach(new ForEachEvaluator() { + int counter = 0; + + @Override + public boolean evaluate(final Integer value) { + final int expected = values[counter++]; + countHolder.incrementAndGet(); + assertEquals(expected, value.intValue()); + return true; + } + + }, IterationDirection.FORWARD); + + assertEquals(4, countHolder.get()); + } + + @Test + public void testIterateForwardsAfterFull() { + final RingBuffer ringBuffer = new RingBuffer<>(10); + + for (int i = 0; i < 12; i++) { + ringBuffer.add(i); + } + + final int[] values = new int[]{3, 5, 20, 7}; + for (final int v : values) { + ringBuffer.add(v); + } + + ringBuffer.forEach(new ForEachEvaluator() { + int counter = 0; + + @Override + public boolean evaluate(final Integer value) { + if (counter < 6) { + assertEquals(counter + 6, value.intValue()); + } else { + final int expected = values[counter - 6]; + assertEquals(expected, value.intValue()); + } + + counter++; + return true; + } + + }, IterationDirection.FORWARD); + } + + @Test + public void testIterateBackwards() { + final RingBuffer ringBuffer = new RingBuffer<>(10); + + final int[] values = new int[]{3, 5, 20, 7}; + for (final int v : values) { + ringBuffer.add(v); + } + + final AtomicInteger countHolder = new AtomicInteger(0); + ringBuffer.forEach(new ForEachEvaluator() { + int counter = 0; + + @Override + public boolean evaluate(final Integer value) { + final int index = values.length - 1 - counter; + final int expected = values[index]; + countHolder.incrementAndGet(); + + assertEquals(expected, value.intValue()); + counter++; + return true; + } + + }, IterationDirection.BACKWARD); + + assertEquals(4, countHolder.get()); + } + + @Test + public void testIterateBackwardsAfterFull() { + final RingBuffer ringBuffer = new RingBuffer<>(10); + + for (int i = 0; i < 12; i++) { + ringBuffer.add(i); + } + + final int[] values = new int[]{3, 5, 20, 7}; + for (final int v : values) { + ringBuffer.add(v); + } + + ringBuffer.forEach(new ForEachEvaluator() { + int counter = 0; + + @Override + public boolean evaluate(final Integer value) { + if (counter < values.length) { + final int index = values.length - 1 - counter; + final int expected = values[index]; + + assertEquals(expected, value.intValue()); + counter++; + } + + return true; + } + + }, IterationDirection.BACKWARD); + } +} diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java new file mode 100644 index 0000000000..4b2c0d587b --- /dev/null +++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestSynchronousFileWatcher.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util.timebuffer; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.nio.file.StandardCopyOption; + +import org.junit.Test; + +import org.apache.nifi.io.MD5SumMonitor; +import org.apache.nifi.io.SynchronousFileWatcher; +import org.apache.nifi.io.UpdateMonitor; + +public class TestSynchronousFileWatcher { + + @Test + public void testIt() throws UnsupportedEncodingException, IOException, InterruptedException { + final Path path = Paths.get("target/1.txt"); + Files.copy(new ByteArrayInputStream("Hello, World!".getBytes("UTF-8")), path, StandardCopyOption.REPLACE_EXISTING); + final UpdateMonitor monitor = new MD5SumMonitor(); + + final SynchronousFileWatcher watcher = new SynchronousFileWatcher(path, monitor, 10L); + assertFalse(watcher.checkAndReset()); + Thread.sleep(30L); + assertFalse(watcher.checkAndReset()); + + final FileOutputStream fos = new FileOutputStream(path.toFile()); + try { + fos.write("Good-bye, World!".getBytes("UTF-8")); + fos.getFD().sync(); + } finally { + fos.close(); + } + + assertTrue(watcher.checkAndReset()); + assertFalse(watcher.checkAndReset()); + + Thread.sleep(30L); + assertFalse(watcher.checkAndReset()); + } +} diff --git a/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java new file mode 100644 index 0000000000..39ca3302f8 --- /dev/null +++ b/commons/nifi-utils/src/test/java/org/apache/nifi/util/timebuffer/TestTimedBuffer.java @@ -0,0 +1,106 @@ +/* + * 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.util.timebuffer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.util.concurrent.TimeUnit; + +import org.junit.Test; + +public class TestTimedBuffer { + + @Test + public void testAgesOff() throws InterruptedException { + final LongEntityAccess access = new LongEntityAccess(); + final TimedBuffer buffer = new TimedBuffer<>(TimeUnit.SECONDS, 2, access); + + buffer.add(new TimestampedLong(1000000L)); + TimestampedLong aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L); + assertEquals(1000000L, aggregate.getValue().longValue()); + Thread.sleep(1000L); + aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L); + assertEquals(1000000L, aggregate.getValue().longValue()); + Thread.sleep(1500L); + aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L); + assertNull(aggregate); + } + + @Test + public void testAggregation() throws InterruptedException { + final LongEntityAccess access = new LongEntityAccess(); + final TimedBuffer buffer = new TimedBuffer<>(TimeUnit.SECONDS, 2, access); + + buffer.add(new TimestampedLong(1000000L)); + buffer.add(new TimestampedLong(1000000L)); + buffer.add(new TimestampedLong(25000L)); + + TimestampedLong aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L); + assertEquals(2025000L, aggregate.getValue().longValue()); + Thread.sleep(1000L); + aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L); + assertEquals(2025000L, aggregate.getValue().longValue()); + Thread.sleep(1500L); + aggregate = buffer.getAggregateValue(System.currentTimeMillis() - 30000L); + assertNull(aggregate); + } + + private static class TimestampedLong { + + private final Long value; + private final long timestamp = System.currentTimeMillis(); + + public TimestampedLong(final Long value) { + this.value = value; + } + + public Long getValue() { + return value; + } + + public long getTimestamp() { + return timestamp; + } + } + + private static class LongEntityAccess implements EntityAccess { + + @Override + public TimestampedLong aggregate(TimestampedLong oldValue, TimestampedLong toAdd) { + if (oldValue == null && toAdd == null) { + return new TimestampedLong(0L); + } else if (oldValue == null) { + return toAdd; + } else if (toAdd == null) { + return oldValue; + } + + return new TimestampedLong(oldValue.getValue().longValue() + toAdd.getValue().longValue()); + } + + @Override + public TimestampedLong createNew() { + return new TimestampedLong(0L); + } + + @Override + public long getTimestamp(TimestampedLong entity) { + return entity == null ? 0L : entity.getTimestamp(); + } + } +} diff --git a/commons/nifi-web-utils/pom.xml b/commons/nifi-web-utils/pom.xml new file mode 100644 index 0000000000..434e1a3d39 --- /dev/null +++ b/commons/nifi-web-utils/pom.xml @@ -0,0 +1,56 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-web-utils + 0.0.1-SNAPSHOT + NiFi Web Utils + + + + org.apache.nifi + nifi-security-utils + 0.0.1-SNAPSHOT + + + commons-codec + commons-codec + 1.10 + + + com.sun.jersey + jersey-client + 1.18.2 + + + com.sun.jersey + jersey-json + 1.18.2 + + + javax.servlet + javax.servlet-api + 3.1.0 + provided + + + diff --git a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java new file mode 100644 index 0000000000..8c0b1f4363 --- /dev/null +++ b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ClientUtils.java @@ -0,0 +1,132 @@ +/* + * 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.web.util; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientHandlerException; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.UniformInterfaceException; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.core.util.MultivaluedMapImpl; +import java.net.URI; +import java.util.Map; +import javax.ws.rs.core.MediaType; + +/** + * + */ +public class ClientUtils { + + private final Client client; + + public ClientUtils(Client client) { + this.client = client; + } + + /** + * Gets the content at the specified URI. + * + * @param uri + * @return + * @throws ClientHandlerException + * @throws UniformInterfaceException + */ + public ClientResponse get(final URI uri) throws ClientHandlerException, UniformInterfaceException { + return get(uri, null); + } + + /** + * Gets the content at the specified URI using the given query parameters. + * + * @param uri + * @param queryParams + * @return + * @throws ClientHandlerException + * @throws UniformInterfaceException + */ + public ClientResponse get(final URI uri, final Map queryParams) throws ClientHandlerException, UniformInterfaceException { + // perform the request + WebResource webResource = client.resource(uri); + if (queryParams != null) { + for (final Map.Entry queryEntry : queryParams.entrySet()) { + webResource = webResource.queryParam(queryEntry.getKey(), queryEntry.getValue()); + } + } + + return webResource.accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + } + + /** + * Performs a POST using the specified url and entity body. + * + * @param uri + * @param entity + * @return + */ + public ClientResponse post(URI uri, Object entity) throws ClientHandlerException, UniformInterfaceException { + // get the resource + WebResource.Builder resourceBuilder = client.resource(uri).accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_JSON); + + // include the request entity + if (entity != null) { + resourceBuilder = resourceBuilder.entity(entity); + } + + // perform the request + return resourceBuilder.post(ClientResponse.class); + } + + /** + * Performs a POST using the specified url and form data. + * + * @param uri + * @param formData + * @return + */ + public ClientResponse post(URI uri, Map formData) throws ClientHandlerException, UniformInterfaceException { + // convert the form data + MultivaluedMapImpl entity = new MultivaluedMapImpl(); + for (String key : formData.keySet()) { + entity.add(key, formData.get(key)); + } + + // get the resource + WebResource.Builder resourceBuilder = client.resource(uri).accept(MediaType.APPLICATION_JSON).type(MediaType.APPLICATION_FORM_URLENCODED); + + // add the form data if necessary + if (!entity.isEmpty()) { + resourceBuilder = resourceBuilder.entity(entity); + } + + // perform the request + return resourceBuilder.post(ClientResponse.class); + } + + /** + * Performs a HEAD request to the specified URI. + * + * @param uri + * @return + * @throws ClientHandlerException + * @throws UniformInterfaceException + */ + public ClientResponse head(final URI uri) throws ClientHandlerException, UniformInterfaceException { + // perform the request + WebResource webResource = client.resource(uri); + return webResource.head(); + } +} diff --git a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java new file mode 100644 index 0000000000..4e7f5b6dfd --- /dev/null +++ b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/ObjectMapperResolver.java @@ -0,0 +1,48 @@ +/* + * 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.web.util; + +import javax.ws.rs.ext.ContextResolver; +import javax.ws.rs.ext.Provider; +import org.codehaus.jackson.map.AnnotationIntrospector; +import org.codehaus.jackson.map.DeserializationConfig; +import org.codehaus.jackson.map.ObjectMapper; +import org.codehaus.jackson.map.SerializationConfig; +import org.codehaus.jackson.map.annotate.JsonSerialize.Inclusion; +import org.codehaus.jackson.xc.JaxbAnnotationIntrospector; + +@Provider +public class ObjectMapperResolver implements ContextResolver { + + private final ObjectMapper mapper; + + public ObjectMapperResolver() throws Exception { + mapper = new ObjectMapper(); + + final AnnotationIntrospector jaxbIntrospector = new JaxbAnnotationIntrospector(); + final SerializationConfig serializationConfig = mapper.getSerializationConfig(); + final DeserializationConfig deserializationConfig = mapper.getDeserializationConfig(); + + mapper.setSerializationConfig(serializationConfig.withSerializationInclusion(Inclusion.NON_NULL).withAnnotationIntrospector(jaxbIntrospector)); + mapper.setDeserializationConfig(deserializationConfig.withAnnotationIntrospector(jaxbIntrospector)); + } + + @Override + public ObjectMapper getContext(Class objectType) { + return mapper; + } +} diff --git a/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java new file mode 100644 index 0000000000..587b3d8eac --- /dev/null +++ b/commons/nifi-web-utils/src/main/java/org/apache/nifi/web/util/WebUtils.java @@ -0,0 +1,198 @@ +/* + * 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.web.util; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; +import java.security.cert.Certificate; +import java.security.cert.CertificateParsingException; +import java.security.cert.X509Certificate; +import java.util.List; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import javax.net.ssl.HostnameVerifier; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.net.ssl.SSLSession; + +import org.apache.nifi.security.util.CertificateUtils; + +import org.apache.commons.codec.DecoderException; +import org.apache.commons.codec.binary.Hex; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.config.DefaultClientConfig; +import com.sun.jersey.api.json.JSONConfiguration; +import com.sun.jersey.client.urlconnection.HTTPSProperties; + +/** + * Common utilities related to web development. + * + * @author unattributed + */ +public final class WebUtils { + + private static Logger logger = LoggerFactory.getLogger(WebUtils.class); + + final static ReadWriteLock lock = new ReentrantReadWriteLock(); + + private WebUtils() { + } + + /** + * Creates a client for non-secure requests. The client will be created + * using the given configuration. Additionally, the client will be + * automatically configured for JSON serialization/deserialization. + * + * @param config client configuration + * + * @return a Client instance + */ + public static Client createClient(final ClientConfig config) { + return createClientHelper(config, null); + } + + /** + * Creates a client for secure requests. The client will be created using + * the given configuration and security context. Additionally, the client + * will be automatically configured for JSON serialization/deserialization. + * + * @param config client configuration + * @param ctx security context + * + * @return a Client instance + */ + public static Client createClient(final ClientConfig config, final SSLContext ctx) { + return createClientHelper(config, ctx); + } + + /** + * A helper method for creating clients. The client will be created using + * the given configuration and security context. Additionally, the client + * will be automatically configured for JSON serialization/deserialization. + * + * @param config client configuration + * @param ctx security context, which may be null for non-secure client + * creation + * + * @return a Client instance + */ + private static Client createClientHelper(final ClientConfig config, final SSLContext ctx) { + + final ClientConfig finalConfig = (config == null) ? new DefaultClientConfig() : config; + + if (ctx != null && StringUtils.isBlank((String) finalConfig.getProperty(HTTPSProperties.PROPERTY_HTTPS_PROPERTIES))) { + + // custom hostname verifier that checks subject alternative names against the hostname of the URI + final HostnameVerifier hostnameVerifier = new HostnameVerifier() { + @Override + public boolean verify(final String hostname, final SSLSession ssls) { + + try { + for (final Certificate peerCertificate : ssls.getPeerCertificates()) { + if (peerCertificate instanceof X509Certificate) { + final X509Certificate x509Cert = (X509Certificate) peerCertificate; + final List subjectAltNames = CertificateUtils.getSubjectAlternativeNames(x509Cert); + if (subjectAltNames.contains(hostname.toLowerCase())) { + return true; + } + } + } + } catch (final SSLPeerUnverifiedException | CertificateParsingException ex) { + logger.warn("Hostname Verification encountered exception verifying hostname due to: " + ex, ex); + } + + return false; + } + }; + + finalConfig.getProperties().put(HTTPSProperties.PROPERTY_HTTPS_PROPERTIES, new HTTPSProperties(hostnameVerifier, ctx)); + } + + finalConfig.getFeatures().put(JSONConfiguration.FEATURE_POJO_MAPPING, Boolean.TRUE); + finalConfig.getClasses().add(ObjectMapperResolver.class); + + // web client for restful request + return Client.create(finalConfig); + + } + + /** + * Serializes the given object to hexadecimal. Serialization uses Java's + * native serialization mechanism, the ObjectOutputStream. + * + * @param obj an object + * @return the serialized object as hex + */ + public static String serializeObjectToHex(final Serializable obj) { + + final ByteArrayOutputStream serializedObj = new ByteArrayOutputStream(); + + // IOException can never be thrown because we are serializing to an in memory byte array + try { + final ObjectOutputStream oos = new ObjectOutputStream(serializedObj); + oos.writeObject(obj); + oos.close(); + } catch (final IOException ioe) { + throw new RuntimeException(ioe); + } + + logger.debug(String.format("Serialized object '%s' size: %d", obj, serializedObj.size())); + + // hex encode the binary + return new String(Hex.encodeHex(serializedObj.toByteArray(), /* tolowercase */ true)); + } + + /** + * Deserializes a Java serialized, hex-encoded string into a Java object. + * This method is the inverse of the serializeObjectToHex method in this + * class. + * + * @param hexEncodedObject a string + * @return the object + * @throws ClassNotFoundException if the class could not be found + */ + public static Serializable deserializeHexToObject(final String hexEncodedObject) throws ClassNotFoundException { + + // decode the hex encoded object + byte[] serializedObj; + try { + serializedObj = Hex.decodeHex(hexEncodedObject.toCharArray()); + } catch (final DecoderException de) { + throw new IllegalArgumentException(de); + } + + // IOException can never be thrown because we are deserializing from an in memory byte array + try { + // deserialize bytes into object + ObjectInputStream ois = new ObjectInputStream(new ByteArrayInputStream(serializedObj)); + return (Serializable) ois.readObject(); + } catch (final IOException ioe) { + throw new RuntimeException(ioe); + } + + } +} diff --git a/commons/processor-utilities/pom.xml b/commons/processor-utilities/pom.xml new file mode 100644 index 0000000000..0519b7ffb5 --- /dev/null +++ b/commons/processor-utilities/pom.xml @@ -0,0 +1,47 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-processor-utils + 0.0.1-SNAPSHOT + jar + NiFi Processor Utils + + + + org.apache.nifi + nifi-api + [0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT) + provided + + + org.apache.nifi + nifi-utils + [0.0.1-SNAPSHOT,1.0.0-SNAPSHOT) + + + org.apache.nifi + nifi-security-utils + 0.0.1-SNAPSHOT + + + diff --git a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java new file mode 100644 index 0000000000..1f77093006 --- /dev/null +++ b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/FlowFileFilters.java @@ -0,0 +1,65 @@ +/* + * 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.processor.util; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.FlowFileFilter; + +public class FlowFileFilters { + + /** + * Returns a new {@link FlowFileFilter} that will pull FlowFiles until the + * maximum file size has been reached, or the maximum FlowFile Count was + * been reached (this is important because FlowFiles may be 0 bytes!). If + * the first FlowFile exceeds the max size, the FlowFile will be selected + * and no other FlowFile will be. + * + * @param maxSize the maximum size of the group of FlowFiles + * @param unit the unit of the maxSize argument + * @param maxCount the maximum number of FlowFiles to pull + * @return + */ + public static FlowFileFilter newSizeBasedFilter(final double maxSize, final DataUnit unit, final int maxCount) { + final double maxBytes = DataUnit.B.convert(maxSize, unit); + + return new FlowFileFilter() { + int count = 0; + long size = 0L; + + @Override + public FlowFileFilterResult filter(final FlowFile flowFile) { + if (count == 0) { + count++; + size += flowFile.getSize(); + + return FlowFileFilterResult.ACCEPT_AND_CONTINUE; + } + + if ((size + flowFile.getSize() > maxBytes) || (count + 1 > maxCount)) { + return FlowFileFilterResult.REJECT_AND_TERMINATE; + } + + count++; + size += flowFile.getSize(); + return FlowFileFilterResult.ACCEPT_AND_CONTINUE; + } + + }; + } + +} diff --git a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java new file mode 100644 index 0000000000..0d66df51f8 --- /dev/null +++ b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/SSLProperties.java @@ -0,0 +1,226 @@ +/* + * 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.processor.util; + +import java.io.File; +import java.io.IOException; +import java.net.MalformedURLException; +import java.security.KeyManagementException; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.security.util.CertificateUtils; +import org.apache.nifi.security.util.KeystoreType; +import org.apache.nifi.security.util.SslContextFactory; +import org.apache.nifi.security.util.SslContextFactory.ClientAuth; + +public class SSLProperties { + + public static final PropertyDescriptor TRUSTSTORE = new PropertyDescriptor.Builder() + .name("Truststore Filename") + .description("The fully-qualified filename of the Truststore") + .defaultValue(null) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .sensitive(false) + .build(); + + public static final PropertyDescriptor TRUSTSTORE_TYPE = new PropertyDescriptor.Builder() + .name("Truststore Type") + .description("The Type of the Truststore. Either JKS or PKCS12") + .allowableValues("JKS", "PKCS12") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .defaultValue(null) + .sensitive(false) + .build(); + + public static final PropertyDescriptor TRUSTSTORE_PASSWORD = new PropertyDescriptor.Builder() + .name("Truststore Password") + .description("The password for the Truststore") + .defaultValue(null) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(true) + .build(); + + public static final PropertyDescriptor KEYSTORE = new PropertyDescriptor.Builder() + .name("Keystore Filename") + .description("The fully-qualified filename of the Keystore") + .defaultValue(null) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .sensitive(false) + .build(); + + public static final PropertyDescriptor KEYSTORE_TYPE = new PropertyDescriptor.Builder() + .name("Keystore Type") + .description("The Type of the Keystore") + .allowableValues("JKS", "PKCS12") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(false) + .build(); + + public static final PropertyDescriptor KEYSTORE_PASSWORD = new PropertyDescriptor.Builder() + .name("Keystore Password") + .defaultValue(null) + .description("The password for the Keystore") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(true) + .build(); + + public static Collection validateStore(final Map properties) { + final Collection results = new ArrayList<>(); + results.addAll(validateStore(properties, KeystoreValidationGroup.KEYSTORE)); + results.addAll(validateStore(properties, KeystoreValidationGroup.TRUSTSTORE)); + return results; + } + + public static Collection validateStore(final Map properties, final KeystoreValidationGroup keyStoreOrTrustStore) { + final Collection results = new ArrayList<>(); + + final String filename; + final String password; + final String type; + + if (keyStoreOrTrustStore == KeystoreValidationGroup.KEYSTORE) { + filename = properties.get(KEYSTORE); + password = properties.get(KEYSTORE_PASSWORD); + type = properties.get(KEYSTORE_TYPE); + } else { + filename = properties.get(TRUSTSTORE); + password = properties.get(TRUSTSTORE_PASSWORD); + type = properties.get(TRUSTSTORE_TYPE); + } + + final String keystoreDesc = (keyStoreOrTrustStore == KeystoreValidationGroup.KEYSTORE) ? "Keystore" : "Truststore"; + + final int nulls = countNulls(filename, password, type); + if (nulls != 3 && nulls != 0) { + results.add(new ValidationResult.Builder().valid(false).explanation("Must set either 0 or 3 properties for " + keystoreDesc).subject(keystoreDesc + " Properties").build()); + } else if (nulls == 0) { + // all properties were filled in. + final File file = new File(filename); + if (!file.exists() || !file.canRead()) { + results.add(new ValidationResult.Builder().valid(false).subject(keystoreDesc + " Properties").explanation("Cannot access file " + file.getAbsolutePath()).build()); + } else { + try { + final boolean storeValid = CertificateUtils.isStoreValid(file.toURI().toURL(), KeystoreType.valueOf(type), password.toCharArray()); + if (!storeValid) { + results.add(new ValidationResult.Builder().subject(keystoreDesc + " Properties").valid(false).explanation("Invalid KeyStore Password or Type specified for file " + filename).build()); + } + } catch (MalformedURLException e) { + results.add(new ValidationResult.Builder().subject(keystoreDesc + " Properties").valid(false).explanation("Malformed URL from file: " + e).build()); + } + } + } + + return results; + } + + private static int countNulls(Object... objects) { + int count = 0; + for (final Object x : objects) { + if (x == null) { + count++; + } + } + + return count; + } + + public static enum KeystoreValidationGroup { + + KEYSTORE, TRUSTSTORE + } + + public static List getKeystoreDescriptors(final boolean required) { + final List descriptors = new ArrayList<>(); + for (final PropertyDescriptor descriptor : KEYSTORE_DESCRIPTORS) { + final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder().fromPropertyDescriptor(descriptor).required(required); + if (required && descriptor.getName().equals(KEYSTORE_TYPE.getName())) { + builder.defaultValue("JKS"); + } + descriptors.add(builder.build()); + } + + return descriptors; + } + + public static List getTruststoreDescriptors(final boolean required) { + final List descriptors = new ArrayList<>(); + for (final PropertyDescriptor descriptor : TRUSTSTORE_DESCRIPTORS) { + final PropertyDescriptor.Builder builder = new PropertyDescriptor.Builder().fromPropertyDescriptor(descriptor).required(required); + if (required && descriptor.getName().equals(TRUSTSTORE_TYPE.getName())) { + builder.defaultValue("JKS"); + } + descriptors.add(builder.build()); + } + + return descriptors; + } + + public static SSLContext createSSLContext(final ProcessContext context, final ClientAuth clientAuth) + throws UnrecoverableKeyException, KeyManagementException, KeyStoreException, NoSuchAlgorithmException, CertificateException, IOException { + final String keystoreFile = context.getProperty(KEYSTORE).getValue(); + if (keystoreFile == null) { + return SslContextFactory.createTrustSslContext( + context.getProperty(TRUSTSTORE).getValue(), + context.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(), + context.getProperty(TRUSTSTORE_TYPE).getValue()); + } else { + final String truststoreFile = context.getProperty(TRUSTSTORE).getValue(); + if (truststoreFile == null) { + return SslContextFactory.createSslContext( + context.getProperty(KEYSTORE).getValue(), + context.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(), + context.getProperty(KEYSTORE_TYPE).getValue()); + } else { + return SslContextFactory.createSslContext( + context.getProperty(KEYSTORE).getValue(), + context.getProperty(KEYSTORE_PASSWORD).getValue().toCharArray(), + context.getProperty(KEYSTORE_TYPE).getValue(), + context.getProperty(TRUSTSTORE).getValue(), + context.getProperty(TRUSTSTORE_PASSWORD).getValue().toCharArray(), + context.getProperty(TRUSTSTORE_TYPE).getValue(), + clientAuth); + } + } + } + + private static final Set KEYSTORE_DESCRIPTORS = new HashSet<>(); + private static final Set TRUSTSTORE_DESCRIPTORS = new HashSet<>(); + + static { + KEYSTORE_DESCRIPTORS.add(KEYSTORE); + KEYSTORE_DESCRIPTORS.add(KEYSTORE_TYPE); + KEYSTORE_DESCRIPTORS.add(KEYSTORE_PASSWORD); + + TRUSTSTORE_DESCRIPTORS.add(TRUSTSTORE); + TRUSTSTORE_DESCRIPTORS.add(TRUSTSTORE_TYPE); + TRUSTSTORE_DESCRIPTORS.add(TRUSTSTORE_PASSWORD); + } +} diff --git a/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java new file mode 100644 index 0000000000..10748fe0af --- /dev/null +++ b/commons/processor-utilities/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -0,0 +1,544 @@ +/* + * 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.processor.util; + +import java.io.File; +import java.net.URI; +import java.net.URL; +import java.nio.charset.Charset; +import java.nio.charset.UnsupportedCharsetException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import java.util.regex.Pattern; + +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.expression.AttributeExpression.ResultType; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.util.FormatUtils; + +public class StandardValidators { + + // + // + // STATICALLY DEFINED VALIDATORS + // + // + public static final Validator ATTRIBUTE_KEY_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final ValidationResult.Builder builder = new ValidationResult.Builder(); + builder.subject(subject).input(input); + + try { + FlowFile.KeyValidator.validateKey(input); + builder.valid(true); + } catch (final IllegalArgumentException e) { + builder.valid(false).explanation(e.getMessage()); + } + + return builder.build(); + } + }; + + public static final Validator ATTRIBUTE_KEY_PROPERTY_NAME_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final ValidationResult.Builder builder = new ValidationResult.Builder(); + builder.subject("Property Name").input(subject); + + try { + FlowFile.KeyValidator.validateKey(subject); + builder.valid(true); + } catch (final IllegalArgumentException e) { + builder.valid(false).explanation(e.getMessage()); + } + + return builder.build(); + } + }; + + public static final Validator POSITIVE_INTEGER_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + String reason = null; + try { + final int intVal = Integer.parseInt(value); + + if (intVal <= 0) { + reason = "not a positive value"; + } + } catch (final NumberFormatException e) { + reason = "not a valid integer"; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + }; + + public static final Validator POSITIVE_LONG_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + String reason = null; + try { + final long longVal = Long.parseLong(value); + + if (longVal <= 0) { + reason = "not a positive value"; + } + } catch (final NumberFormatException e) { + reason = "not a valid 64-bit integer"; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + }; + + public static final Validator PORT_VALIDATOR = createLongValidator(1, 65535, true); + + public static final Validator NON_EMPTY_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + return new ValidationResult.Builder().subject(subject).input(value).valid(value != null && !value.isEmpty()).explanation(subject + " cannot be empty").build(); + } + }; + + public static final Validator BOOLEAN_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + final boolean valid = "true".equalsIgnoreCase(value) || "false".equalsIgnoreCase(value); + final String explanation = valid ? null : "Value must be 'true' or 'false'"; + return new ValidationResult.Builder().subject(subject).input(value).valid(valid).explanation(explanation).build(); + } + }; + + public static final Validator INTEGER_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + String reason = null; + try { + Integer.parseInt(value); + } catch (final NumberFormatException e) { + reason = "not a valid integer"; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + }; + + public static final Validator LONG_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + String reason = null; + try { + Long.parseLong(value); + } catch (final NumberFormatException e) { + reason = "not a valid Long"; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + }; + + public static final Validator NON_NEGATIVE_INTEGER_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + String reason = null; + try { + final int intVal = Integer.parseInt(value); + + if (intVal < 0) { + reason = "value is negative"; + } + } catch (final NumberFormatException e) { + reason = "value is not a valid integer"; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + }; + + public static final Validator CHARACTER_SET_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + String reason = null; + try { + if (!Charset.isSupported(value)) { + reason = "Character Set is not supported by this JVM."; + } + } catch (final UnsupportedCharsetException uce) { + reason = "Character Set is not supported by this JVM."; + } catch (final IllegalArgumentException iae) { + reason = "Character Set value cannot be null."; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + }; + + /** + * URL Validator that does not allow the Expression Language to be used + */ + public static final Validator URL_VALIDATOR = createURLValidator(); + + public static final Validator URI_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + try { + new URI(input); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Valid URI").valid(true).build(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Not a valid URI").valid(false).build(); + } + } + }; + + public static final Validator REGULAR_EXPRESSION_VALIDATOR = createRegexValidator(0, Integer.MAX_VALUE, false); + + public static final Validator ATTRIBUTE_EXPRESSION_LANGUAGE_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + try { + context.newExpressionLanguageCompiler().compile(input); + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(e.getMessage()).build(); + } + } + + }; + + public static final Validator TIME_PERIOD_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if (input == null) { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); + } + if (Pattern.compile(FormatUtils.TIME_DURATION_REGEX).matcher(input.toLowerCase()).matches()) { + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } else { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Must be of format where is a non-negative integer and TimeUnit is a supported Time Unit, such as: nanos, millis, secs, mins, hrs, days").build(); + } + } + }; + + public static final Validator DATA_SIZE_VALIDATOR = new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if (input == null) { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Data Size cannot be null").build(); + } + if (Pattern.compile(DataUnit.DATA_SIZE_REGEX).matcher(input.toUpperCase()).matches()) { + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } else { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Must be of format where is a non-negative integer and is a supported Data Unit, such as: B, KB, MB, GB, TB").build(); + } + } + }; + + public static final Validator FILE_EXISTS_VALIDATOR = new FileExistsValidator(true); + + // + // + // FACTORY METHODS FOR VALIDATORS + // + // + public static Validator createDirectoryExistsValidator(final boolean allowExpressionLanguage, final boolean createDirectoryIfMissing) { + return new DirectoryExistsValidator(allowExpressionLanguage, createDirectoryIfMissing); + } + + private static Validator createURLValidator() { + return new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + try { + final String evaluatedInput = context.newPropertyValue(input).evaluateAttributeExpressions().getValue(); + new URL(evaluatedInput); + return new ValidationResult.Builder().subject(subject).input(input).explanation("Valid URL").valid(true).build(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(input).explanation("Not a valid URL").valid(false).build(); + } + } + }; + } + + public static Validator createTimePeriodValidator(final long minTime, final TimeUnit minTimeUnit, final long maxTime, final TimeUnit maxTimeUnit) { + return new TimePeriodValidator(minTime, minTimeUnit, maxTime, maxTimeUnit); + } + + public static Validator createAttributeExpressionLanguageValidator(final ResultType expectedResultType) { + return createAttributeExpressionLanguageValidator(expectedResultType, true); + } + + public static Validator createRegexMatchingValidator(final Pattern pattern) { + return new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final boolean matches = pattern.matcher(input).matches(); + return new ValidationResult.Builder() + .input(input) + .subject(subject) + .valid(matches) + .explanation(matches ? null : "Value does not match regular expression: " + pattern.pattern()) + .build(); + } + }; + } + + /** + * Creates a @{link Validator} that ensure that a value is a valid Java + * Regular Expression with at least minCapturingGroups + * capturing groups and at most maxCapturingGroups capturing + * groups. If supportAttributeExpressionLanguage is set to + * true, the value may also include the Expression Language, + * but the result of evaluating the Expression Language will be applied + * before the Regular Expression is performed. In this case, the Expression + * Language will not support FlowFile Attributes but only System/JVM + * Properties + * + * @param minCapturingGroups + * @param maxCapturingGroups + * @param supportAttributeExpressionLanguage + * @return + */ + public static Validator createRegexValidator(final int minCapturingGroups, final int maxCapturingGroups, final boolean supportAttributeExpressionLanguage) { + return new Validator() { + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + try { + final String substituted; + if (supportAttributeExpressionLanguage) { + try { + substituted = context.newPropertyValue(value).evaluateAttributeExpressions().getValue(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation("Failed to evaluate the Attribute Expression Language due to " + e.toString()).build(); + } + } else { + substituted = value; + } + + final Pattern pattern = Pattern.compile(substituted); + final int numGroups = pattern.matcher("").groupCount(); + if (numGroups < minCapturingGroups || numGroups > maxCapturingGroups) { + return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation("RegEx is required to have between " + minCapturingGroups + " and " + maxCapturingGroups + " Capturing Groups but has " + numGroups).build(); + } + + return new ValidationResult.Builder().subject(subject).input(value).valid(true).build(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(value).valid(false).explanation("Not a valid Java Regular Expression").build(); + } + + } + }; + } + + public static Validator createAttributeExpressionLanguageValidator(final ResultType expectedResultType, final boolean allowExtraCharacters) { + return new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final String syntaxError = context.newExpressionLanguageCompiler().validateExpression(input, allowExtraCharacters); + if (syntaxError != null) { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation(syntaxError).build(); + } + + final ResultType resultType = allowExtraCharacters ? ResultType.STRING : context.newExpressionLanguageCompiler().getResultType(input); + if (!resultType.equals(expectedResultType)) { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Expected Attribute Query to return type " + expectedResultType + " but query returns type " + resultType).build(); + } + + return new ValidationResult.Builder().subject(subject).input(input).valid(true).build(); + } + }; + } + + public static Validator createLongValidator(final long minimum, final long maximum, final boolean inclusive) { + return new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + String reason = null; + try { + final long longVal = Long.parseLong(input); + if (longVal < minimum || (!inclusive && longVal == minimum) | longVal > maximum || (!inclusive && longVal == maximum)) { + reason = "Value must be between " + minimum + " and " + maximum + " (" + (inclusive ? "inclusive" : "exclusive") + ")"; + } + } catch (final NumberFormatException e) { + reason = "not a valid integer"; + } + + return new ValidationResult.Builder().subject(subject).input(input).explanation(reason).valid(reason == null).build(); + } + + }; + } + + // + // + // SPECIFIC VALIDATOR IMPLEMENTATIONS THAT CANNOT BE ANONYMOUS CLASSES + // + // + static class TimePeriodValidator implements Validator { + + private final Pattern pattern; + + private final long minNanos; + private final long maxNanos; + + private final String minValueEnglish; + private final String maxValueEnglish; + + public TimePeriodValidator(final long minValue, final TimeUnit minTimeUnit, final long maxValue, final TimeUnit maxTimeUnit) { + pattern = Pattern.compile(FormatUtils.TIME_DURATION_REGEX); + + this.minNanos = TimeUnit.NANOSECONDS.convert(minValue, minTimeUnit); + this.maxNanos = TimeUnit.NANOSECONDS.convert(maxValue, maxTimeUnit); + this.minValueEnglish = minValue + " " + minTimeUnit.toString(); + this.maxValueEnglish = maxValue + " " + maxTimeUnit.toString(); + } + + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if (input == null) { + return new ValidationResult.Builder().subject(subject).input(input).valid(false).explanation("Time Period cannot be null").build(); + } + final String lowerCase = input.toLowerCase(); + final boolean validSyntax = pattern.matcher(lowerCase).matches(); + final ValidationResult.Builder builder = new ValidationResult.Builder(); + if (validSyntax) { + final long nanos = FormatUtils.getTimeDuration(lowerCase, TimeUnit.NANOSECONDS); + + if (nanos < minNanos || nanos > maxNanos) { + builder.subject(subject).input(input).valid(false) + .explanation("Must be in the range of " + minValueEnglish + " to " + maxValueEnglish); + } else { + builder.subject(subject).input(input).valid(true); + } + } else { + builder.subject(subject).input(input).valid(false) + .explanation("Must be of format where is a non-negative integer and TimeUnit is a supported Time Unit, such as: nanos, millis, secs, mins, hrs, days"); + } + return builder.build(); + } + } + + public static class FileExistsValidator implements Validator { + + private final boolean allowEL; + + public FileExistsValidator(final boolean allowExpressionLanguage) { + this.allowEL = allowExpressionLanguage; + } + + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + final String substituted; + if (allowEL) { + try { + substituted = context.newPropertyValue(value).evaluateAttributeExpressions().getValue(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(value).valid(false) + .explanation("Not a valid Expression Language value: " + e.getMessage()).build(); + } + } else { + substituted = value; + } + + final File file = new File(substituted); + final boolean valid = file.exists(); + final String explanation = valid ? null : "File " + file + " does not exist"; + return new ValidationResult.Builder().subject(subject).input(value).valid(valid).explanation(explanation).build(); + } + } + + public static class DirectoryExistsValidator implements Validator { + + private final boolean allowEL; + private final boolean create; + + public DirectoryExistsValidator(final boolean allowExpressionLanguage, final boolean create) { + this.allowEL = allowExpressionLanguage; + this.create = create; + } + + @Override + public ValidationResult validate(final String subject, final String value, final ValidationContext context) { + final String substituted; + if (allowEL) { + try { + substituted = context.newPropertyValue(value).evaluateAttributeExpressions().getValue(); + } catch (final Exception e) { + return new ValidationResult.Builder().subject(subject).input(value).valid(false) + .explanation("Not a valid Expression Language value: " + e.getMessage()).build(); + } + + if (substituted.trim().isEmpty() && !value.trim().isEmpty()) { + // User specified an Expression and nothing more... assume valid. + return new ValidationResult.Builder().subject(subject).input(value).valid(true).build(); + } + } else { + substituted = value; + } + + String reason = null; + try { + final File file = new File(substituted); + if (!file.exists()) { + if (!create) { + reason = "Directory does not exist"; + } else if (!file.mkdirs()) { + reason = "Directory does not exist and could not be created"; + } + } else if (!file.isDirectory()) { + reason = "Path does not point to a directory"; + } + } catch (final Exception e) { + reason = "Value is not a valid directory name"; + } + + return new ValidationResult.Builder().subject(subject).input(value).explanation(reason).valid(reason == null).build(); + } + } + + public static Validator createControllerServiceExistsValidator(final Class serviceClass) { + return new Validator() { + @Override + public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + final ControllerService svc = context.getControllerServiceLookup().getControllerService(input); + + if (svc == null) { + return new ValidationResult.Builder().valid(false).input(input).subject(subject).explanation("No Controller Service exists with this ID").build(); + } + + if (!serviceClass.isAssignableFrom(svc.getClass())) { + return new ValidationResult.Builder().valid(false).input(input).subject(subject).explanation("Controller Service with this ID is of type " + svc.getClass().getName() + " but is expected to be of type " + serviceClass.getName()).build(); + } + + final ValidationContext serviceValidationContext = context.getControllerServiceValidationContext(svc); + final Collection serviceValidationResults = svc.validate(serviceValidationContext); + for (final ValidationResult result : serviceValidationResults) { + if (!result.isValid()) { + return new ValidationResult.Builder().valid(false).input(input).subject(subject).explanation("Controller Service " + input + " is not valid: " + result.getExplanation()).build(); + } + } + + return new ValidationResult.Builder().input(input).subject(subject).valid(true).build(); + } + }; + } +} diff --git a/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java new file mode 100644 index 0000000000..359def2a90 --- /dev/null +++ b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/TestFormatUtils.java @@ -0,0 +1,40 @@ +/* + * 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.processor; + +import static org.junit.Assert.assertEquals; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.util.FormatUtils; + +import org.junit.Test; + +public class TestFormatUtils { + + @Test + public void testParse() { + assertEquals(3, FormatUtils.getTimeDuration("3000 ms", TimeUnit.SECONDS)); + assertEquals(3000, FormatUtils.getTimeDuration("3000 s", TimeUnit.SECONDS)); + assertEquals(0, FormatUtils.getTimeDuration("999 millis", TimeUnit.SECONDS)); + assertEquals(4L * 24L * 60L * 60L * 1000000000L, FormatUtils.getTimeDuration("4 days", TimeUnit.NANOSECONDS)); + assertEquals(24, FormatUtils.getTimeDuration("1 DAY", TimeUnit.HOURS)); + assertEquals(60, FormatUtils.getTimeDuration("1 hr", TimeUnit.MINUTES)); + assertEquals(60, FormatUtils.getTimeDuration("1 Hrs", TimeUnit.MINUTES)); + } + +} diff --git a/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java new file mode 100644 index 0000000000..2ae50c91c7 --- /dev/null +++ b/commons/processor-utilities/src/test/java/org/apache/nifi/processor/util/TestStandardValidators.java @@ -0,0 +1,54 @@ +/* + * 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.processor.util; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; + +import org.junit.Test; + +public class TestStandardValidators { + + @Test + public void testTimePeriodValidator() { + Validator val = StandardValidators.createTimePeriodValidator(1L, TimeUnit.SECONDS, Long.MAX_VALUE, TimeUnit.NANOSECONDS); + ValidationResult vr; + + vr = val.validate("TimePeriodTest", "0 sense made", null); + assertFalse(vr.isValid()); + + vr = val.validate("TimePeriodTest", null, null); + assertFalse(vr.isValid()); + + vr = val.validate("TimePeriodTest", "0 secs", null); + assertFalse(vr.isValid()); + + vr = val.validate("TimePeriodTest", "999 millis", null); + assertFalse(vr.isValid()); + + vr = val.validate("TimePeriodTest", "999999999 nanos", null); + assertFalse(vr.isValid()); + + vr = val.validate("TimePeriodTest", "1 sec", null); + assertTrue(vr.isValid()); + } +} diff --git a/commons/remote-communications-utils/pom.xml b/commons/remote-communications-utils/pom.xml new file mode 100644 index 0000000000..5e5ebc1779 --- /dev/null +++ b/commons/remote-communications-utils/pom.xml @@ -0,0 +1,29 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + remote-communications-utils + 0.0.1-SNAPSHOT + jar + + remote-communications-utils + diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java new file mode 100644 index 0000000000..77c34c93c1 --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/StandardVersionNegotiator.java @@ -0,0 +1,81 @@ +/* + * 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.remote; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +public class StandardVersionNegotiator implements VersionNegotiator { + + private final List versions; + private int curVersion; + + public StandardVersionNegotiator(final int... supportedVersions) { + if (Objects.requireNonNull(supportedVersions).length == 0) { + throw new IllegalArgumentException("At least one version must be supported"); + } + + final List supported = new ArrayList<>(); + for (final int version : supportedVersions) { + supported.add(version); + } + this.versions = Collections.unmodifiableList(supported); + this.curVersion = supportedVersions[0]; + } + + @Override + public int getVersion() { + return curVersion; + } + + @Override + public void setVersion(final int version) throws IllegalArgumentException { + if (!isVersionSupported(version)) { + throw new IllegalArgumentException("Version " + version + " is not supported"); + } + + this.curVersion = version; + } + + @Override + public int getPreferredVersion() { + return versions.get(0); + } + + @Override + public Integer getPreferredVersion(final int maxVersion) { + for (final Integer version : this.versions) { + if (maxVersion >= version) { + return version; + } + } + return null; + } + + @Override + public boolean isVersionSupported(final int version) { + return versions.contains(version); + } + + @Override + public List getSupportedVersions() { + return versions; + } + +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java new file mode 100644 index 0000000000..74f9b3dbdf --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/VersionNegotiator.java @@ -0,0 +1,65 @@ +/* + * 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.remote; + +import java.util.List; + +public interface VersionNegotiator { + + /** + * @return the currently configured Version of this resource + */ + int getVersion(); + + /** + * Sets the version of this resource to the specified version. Only the + * lower byte of the version is relevant. + * + * @param version + * @throws IllegalArgumentException if the given Version is not supported by + * this resource, as is indicated by the {@link #isVersionSupported(int)} + * method + */ + void setVersion(int version) throws IllegalArgumentException; + + /** + * + * @return the Version of this resource that is preferred + */ + int getPreferredVersion(); + + /** + * Gets the preferred version of this resource that is no greater than the + * given maxVersion. If no acceptable version exists that is less than + * maxVersion, then null is returned + * + * @param maxVersion + * @return + */ + Integer getPreferredVersion(int maxVersion); + + /** + * Indicates whether or not the specified version is supported by this + * resource + * + * @param version + * @return + */ + boolean isVersionSupported(int version); + + List getSupportedVersions(); +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java new file mode 100644 index 0000000000..05fd915af1 --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/exception/TransmissionDisabledException.java @@ -0,0 +1,25 @@ +/* + * 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.remote.exception; + +/** + * Indicates that the user disabled transmission while communications were + * taking place with a peer + */ +public class TransmissionDisabledException extends RuntimeException { + +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java new file mode 100644 index 0000000000..71cf894daf --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionInputStream.java @@ -0,0 +1,184 @@ +/* + * 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.remote.io; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.zip.DataFormatException; +import java.util.zip.Inflater; + +public class CompressionInputStream extends InputStream { + + private final InputStream in; + private final Inflater inflater; + + private byte[] compressedBuffer; + private byte[] buffer; + + private int bufferIndex; + private boolean eos = false; // whether or not we've reached the end of stream + private boolean allDataRead = false; // different from eos b/c eos means allDataRead == true && buffer is empty + + private final byte[] fourByteBuffer = new byte[4]; + + public CompressionInputStream(final InputStream in) { + this.in = in; + inflater = new Inflater(); + + buffer = new byte[0]; + compressedBuffer = new byte[0]; + bufferIndex = 1; + } + + private String toHex(final byte[] array) { + final StringBuilder sb = new StringBuilder("0x"); + for (final byte b : array) { + final String hex = Integer.toHexString(b).toUpperCase(); + if (hex.length() == 1) { + sb.append("0"); + } + sb.append(hex); + } + return sb.toString(); + } + + protected void readChunkHeader() throws IOException { + // Ensure that we have a valid SYNC chunk + fillBuffer(fourByteBuffer); + if (!Arrays.equals(CompressionOutputStream.SYNC_BYTES, fourByteBuffer)) { + throw new IOException("Invalid CompressionInputStream. Expected first 4 bytes to be 'SYNC' but were " + toHex(fourByteBuffer)); + } + + // determine the size of the decompressed buffer + fillBuffer(fourByteBuffer); + buffer = new byte[toInt(fourByteBuffer)]; + + // determine the size of the compressed buffer + fillBuffer(fourByteBuffer); + compressedBuffer = new byte[toInt(fourByteBuffer)]; + + bufferIndex = buffer.length; // indicate that buffer is empty + } + + private int toInt(final byte[] data) { + return ((data[0] & 0xFF) << 24) + | ((data[1] & 0xFF) << 16) + | ((data[2] & 0xFF) << 8) + | (data[3] & 0xFF); + } + + protected void bufferAndDecompress() throws IOException { + if (allDataRead) { + eos = true; + return; + } + + readChunkHeader(); + fillBuffer(compressedBuffer); + + inflater.setInput(compressedBuffer); + try { + inflater.inflate(buffer); + } catch (final DataFormatException e) { + throw new IOException(e); + } + inflater.reset(); + + bufferIndex = 0; + final int moreDataByte = in.read(); + if (moreDataByte < 1) { + allDataRead = true; + } else if (moreDataByte > 1) { + throw new IOException("Expected indicator of whether or not more data was to come (-1, 0, or 1) but got " + moreDataByte); + } + } + + private void fillBuffer(final byte[] buffer) throws IOException { + int len; + int bytesLeft = buffer.length; + int bytesRead = 0; + while (bytesLeft > 0 && (len = in.read(buffer, bytesRead, bytesLeft)) > 0) { + bytesLeft -= len; + bytesRead += len; + } + + if (bytesRead < buffer.length) { + throw new EOFException(); + } + } + + private boolean isBufferEmpty() { + return bufferIndex >= buffer.length; + } + + @Override + public int read() throws IOException { + if (eos) { + return -1; + } + + if (isBufferEmpty()) { + bufferAndDecompress(); + } + + if (isBufferEmpty()) { + eos = true; + return -1; + } + + return buffer[bufferIndex++]; + } + + @Override + public int read(final byte[] b) throws IOException { + return read(b, 0, b.length); + } + + @Override + public int read(final byte[] b, final int off, final int len) throws IOException { + if (eos) { + return -1; + } + + if (isBufferEmpty()) { + bufferAndDecompress(); + } + + if (isBufferEmpty()) { + eos = true; + return -1; + } + + final int free = buffer.length - bufferIndex; + final int bytesToTransfer = Math.min(len, free); + System.arraycopy(buffer, bufferIndex, b, off, bytesToTransfer); + bufferIndex += bytesToTransfer; + + return bytesToTransfer; + } + + /** + * Does nothing. Does NOT close underlying InputStream + * @throws java.io.IOException + */ + @Override + public void close() throws IOException { + + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java new file mode 100644 index 0000000000..bc46b0ff7b --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/CompressionOutputStream.java @@ -0,0 +1,147 @@ +/* + * 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.remote.io; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.zip.Deflater; + +public class CompressionOutputStream extends OutputStream { + + public static final byte[] SYNC_BYTES = new byte[]{'S', 'Y', 'N', 'C'}; + + public static final int DEFAULT_COMPRESSION_LEVEL = 1; + public static final int DEFAULT_BUFFER_SIZE = 64 << 10; + public static final int MIN_BUFFER_SIZE = 8 << 10; + + private final OutputStream out; + private final Deflater deflater; + + private final byte[] buffer; + private final byte[] compressed; + + private int bufferIndex = 0; + private boolean dataWritten = false; + + public CompressionOutputStream(final OutputStream outStream) { + this(outStream, DEFAULT_BUFFER_SIZE); + } + + public CompressionOutputStream(final OutputStream outStream, final int bufferSize) { + this(outStream, bufferSize, DEFAULT_COMPRESSION_LEVEL, Deflater.DEFAULT_STRATEGY); + } + + public CompressionOutputStream(final OutputStream outStream, final int bufferSize, final int level, final int strategy) { + if (bufferSize < MIN_BUFFER_SIZE) { + throw new IllegalArgumentException("Buffer size must be at least " + MIN_BUFFER_SIZE); + } + + this.out = outStream; + this.deflater = new Deflater(level); + this.deflater.setStrategy(strategy); + buffer = new byte[bufferSize]; + compressed = new byte[bufferSize + 64]; + } + + /** + * Compresses the currently buffered chunk of data and sends it to the + * output stream + * + * @throws IOException + */ + protected void compressAndWrite() throws IOException { + if (bufferIndex <= 0) { + return; + } + + deflater.setInput(buffer, 0, bufferIndex); + deflater.finish(); + final int compressedBytes = deflater.deflate(compressed); + + writeChunkHeader(compressedBytes); + out.write(compressed, 0, compressedBytes); + + bufferIndex = 0; + deflater.reset(); + } + + private void writeChunkHeader(final int compressedBytes) throws IOException { + // If we have already written data, write out a '1' to indicate that we have more data; when we close + // the stream, we instead write a '0' to indicate that we are finished sending data. + if (dataWritten) { + out.write(1); + } + out.write(SYNC_BYTES); + dataWritten = true; + + writeInt(out, bufferIndex); + writeInt(out, compressedBytes); + } + + private void writeInt(final OutputStream out, final int val) throws IOException { + out.write(val >>> 24); + out.write(val >>> 16); + out.write(val >>> 8); + out.write(val); + } + + protected boolean bufferFull() { + return bufferIndex >= buffer.length; + } + + @Override + public void write(final int b) throws IOException { + buffer[bufferIndex++] = (byte) (b & 0xFF); + if (bufferFull()) { + compressAndWrite(); + } + } + + @Override + public void write(final byte[] b) throws IOException { + write(b, 0, b.length); + } + + @Override + public void write(final byte[] b, final int off, final int len) throws IOException { + int bytesLeft = len; + while (bytesLeft > 0) { + final int free = buffer.length - bufferIndex; + final int bytesThisIteration = Math.min(bytesLeft, free); + System.arraycopy(b, off + len - bytesLeft, buffer, bufferIndex, bytesThisIteration); + bufferIndex += bytesThisIteration; + + bytesLeft -= bytesThisIteration; + if (bufferFull()) { + compressAndWrite(); + } + } + } + + @Override + public void flush() throws IOException { + compressAndWrite(); + super.flush(); + } + + @Override + public void close() throws IOException { + compressAndWrite(); + out.write(0); // indicate that the stream is finished. + out.flush(); + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java new file mode 100644 index 0000000000..e03dfbf131 --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableInputStream.java @@ -0,0 +1,117 @@ +/* + * 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.remote.io; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.nifi.remote.exception.TransmissionDisabledException; + +public class InterruptableInputStream extends InputStream { + + private volatile boolean interrupted = false; + private final InputStream in; + + public InterruptableInputStream(final InputStream in) { + this.in = in; + } + + @Override + public int read() throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + return in.read(); + } + + @Override + public int read(byte[] b) throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + return in.read(b); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + return in.read(b, off, len); + } + + @Override + public int available() throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + return in.available(); + } + + @Override + public void close() throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + in.close(); + } + + @Override + public synchronized void mark(int readlimit) { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + in.mark(readlimit); + } + + @Override + public boolean markSupported() { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + return in.markSupported(); + } + + @Override + public synchronized void reset() throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + in.reset(); + } + + @Override + public long skip(long n) throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + return in.skip(n); + } + + public void interrupt() { + interrupted = true; + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java new file mode 100644 index 0000000000..cba5be620e --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/InterruptableOutputStream.java @@ -0,0 +1,81 @@ +/* + * 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.remote.io; + +import java.io.IOException; +import java.io.OutputStream; + +import org.apache.nifi.remote.exception.TransmissionDisabledException; + +public class InterruptableOutputStream extends OutputStream { + + private final OutputStream out; + private volatile boolean interrupted = false; + + public InterruptableOutputStream(final OutputStream out) { + this.out = out; + } + + @Override + public void write(int b) throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + out.write(b); + } + + @Override + public void write(byte[] b) throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + out.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + out.write(b, off, len); + } + + @Override + public void close() throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + out.close(); + } + + @Override + public void flush() throws IOException { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + out.flush(); + } + + public void interrupt() { + this.interrupted = true; + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java new file mode 100644 index 0000000000..68913bdc85 --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/BufferStateManager.java @@ -0,0 +1,111 @@ +/* + * 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.remote.io.socket; + +import java.nio.ByteBuffer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BufferStateManager { + + private static final Logger logger = LoggerFactory.getLogger(BufferStateManager.class); + + private ByteBuffer buffer; + private Direction direction = Direction.WRITE; + + public BufferStateManager(final ByteBuffer buffer) { + this.buffer = buffer; + } + + public BufferStateManager(final ByteBuffer buffer, final Direction direction) { + this.buffer = buffer; + this.direction = direction; + } + + /** + * Ensures that the buffer is at least as big as the size specified, + * resizing the buffer if necessary. This operation MAY change the direction + * of the buffer. + * + * @param requiredSize + */ + public void ensureSize(final int requiredSize) { + if (buffer.capacity() < requiredSize) { + final ByteBuffer newBuffer = ByteBuffer.allocate(requiredSize); + + // we have to read buffer so make sure the direction is correct. + if (direction == Direction.WRITE) { + buffer.flip(); + } + + // Copy from buffer to newBuffer + newBuffer.put(buffer); + + // Swap the buffers + buffer = newBuffer; + + // the new buffer is ready to be written to + direction = Direction.WRITE; + } + } + + public ByteBuffer prepareForWrite(final int requiredSize) { + ensureSize(requiredSize); + + if (direction == Direction.READ) { + direction = Direction.WRITE; + buffer.position(buffer.limit()); + } + + buffer.limit(buffer.capacity()); + return buffer; + } + + public ByteBuffer prepareForRead(final int requiredSize) { + ensureSize(requiredSize); + + if (direction == Direction.WRITE) { + direction = Direction.READ; + buffer.flip(); + } + + return buffer; + } + + /** + * Clears the contents of the buffer and sets direction to WRITE + */ + public void clear() { + logger.debug("Clearing {}", buffer); + buffer.clear(); + direction = Direction.WRITE; + } + + public void compact() { + final String before = buffer.toString(); + buffer.compact(); + logger.debug("Before compact: {}, after: {}", before, buffer); + direction = Direction.WRITE; + } + + public static enum Direction { + + READ, WRITE; + } + +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java new file mode 100644 index 0000000000..32a3f26bc5 --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelInputStream.java @@ -0,0 +1,157 @@ +/* + * 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.remote.io.socket; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedByInterruptException; +import java.nio.channels.SocketChannel; +import java.util.concurrent.TimeUnit; + +public class SocketChannelInputStream extends InputStream { + + private static final long CHANNEL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS); + private final SocketChannel channel; + private volatile int timeoutMillis = 30000; + + private final ByteBuffer oneByteBuffer = ByteBuffer.allocate(1); + private Byte bufferedByte = null; + + public SocketChannelInputStream(final SocketChannel socketChannel) throws IOException { + // this class expects a non-blocking channel + socketChannel.configureBlocking(false); + this.channel = socketChannel; + } + + public void setTimeout(final int timeoutMillis) { + this.timeoutMillis = timeoutMillis; + } + + @Override + public int read() throws IOException { + if (bufferedByte != null) { + final int retVal = bufferedByte & 0xFF; + bufferedByte = null; + return retVal; + } + + oneByteBuffer.flip(); + oneByteBuffer.clear(); + + final long maxTime = System.currentTimeMillis() + timeoutMillis; + int bytesRead; + do { + bytesRead = channel.read(oneByteBuffer); + if (bytesRead == 0) { + if (System.currentTimeMillis() > maxTime) { + throw new SocketTimeoutException("Timed out reading from socket"); + } + try { + TimeUnit.NANOSECONDS.sleep(CHANNEL_EMPTY_WAIT_NANOS); + } catch (InterruptedException e) { + close(); + Thread.currentThread().interrupt(); // set the interrupt status + throw new ClosedByInterruptException(); // simulate an interrupted blocked read operation + } + } + } while (bytesRead == 0); + + if (bytesRead == -1) { + return -1; + } + oneByteBuffer.flip(); + return oneByteBuffer.get() & 0xFF; + } + + @Override + public int read(final byte[] b) throws IOException { + return read(b, 0, b.length); + } + + @Override + public int read(final byte[] b, final int off, final int len) throws IOException { + if (bufferedByte != null) { + final byte retVal = bufferedByte; + bufferedByte = null; + b[off] = retVal; + return 1; + } + + final ByteBuffer buffer = ByteBuffer.wrap(b, off, len); + + final long maxTime = System.currentTimeMillis() + timeoutMillis; + int bytesRead; + do { + bytesRead = channel.read(buffer); + if (bytesRead == 0) { + if (System.currentTimeMillis() > maxTime) { + throw new SocketTimeoutException("Timed out reading from socket"); + } + try { + TimeUnit.NANOSECONDS.sleep(CHANNEL_EMPTY_WAIT_NANOS); + } catch (InterruptedException e) { + close(); + Thread.currentThread().interrupt(); // set the interrupt status + throw new ClosedByInterruptException(); // simulate an interrupted blocked read operation + } + } + } while (bytesRead == 0); + + return bytesRead; + } + + @Override + public int available() throws IOException { + if (bufferedByte != null) { + return 1; + } + + isDataAvailable(); // attempt to read from socket + return (bufferedByte == null) ? 0 : 1; + } + + public boolean isDataAvailable() throws IOException { + if (bufferedByte != null) { + return true; + } + + oneByteBuffer.flip(); + oneByteBuffer.clear(); + final int bytesRead = channel.read(oneByteBuffer); + if (bytesRead == -1) { + throw new EOFException("Peer has closed the stream"); + } + if (bytesRead > 0) { + oneByteBuffer.flip(); + bufferedByte = oneByteBuffer.get(); + return true; + } + return false; + } + + /** + * Closes the underlying socket channel. + * @throws java.io.IOException + */ + @Override + public void close() throws IOException { + channel.close(); + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java new file mode 100644 index 0000000000..77049ad1ad --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/SocketChannelOutputStream.java @@ -0,0 +1,113 @@ +/* + * 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.remote.io.socket; + +import java.io.IOException; +import java.io.OutputStream; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedByInterruptException; +import java.nio.channels.SocketChannel; +import java.util.concurrent.TimeUnit; + +public class SocketChannelOutputStream extends OutputStream { + + private static final long CHANNEL_FULL_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS); + private final SocketChannel channel; + private volatile int timeout = 30000; + + private final ByteBuffer oneByteBuffer = ByteBuffer.allocate(1); + + public SocketChannelOutputStream(final SocketChannel socketChannel) throws IOException { + // this class expects a non-blocking channel + socketChannel.configureBlocking(false); + this.channel = socketChannel; + } + + public void setTimeout(final int timeoutMillis) { + this.timeout = timeoutMillis; + } + + @Override + public void write(final int b) throws IOException { + oneByteBuffer.flip(); + oneByteBuffer.clear(); + oneByteBuffer.put((byte) b); + oneByteBuffer.flip(); + + final int timeoutMillis = this.timeout; + long maxTime = System.currentTimeMillis() + timeoutMillis; + int bytesWritten; + while (oneByteBuffer.hasRemaining()) { + bytesWritten = channel.write(oneByteBuffer); + if (bytesWritten == 0) { + if (System.currentTimeMillis() > maxTime) { + throw new SocketTimeoutException("Timed out writing to socket"); + } + try { + TimeUnit.NANOSECONDS.sleep(CHANNEL_FULL_WAIT_NANOS); + } catch (InterruptedException e) { + close(); + Thread.currentThread().interrupt(); // set the interrupt status + throw new ClosedByInterruptException(); // simulate an interrupted blocked write operation + } + } else { + return; + } + } + } + + @Override + public void write(final byte[] b) throws IOException { + write(b, 0, b.length); + } + + @Override + public void write(final byte[] b, final int off, final int len) throws IOException { + final ByteBuffer buffer = ByteBuffer.wrap(b, off, len); + + final int timeoutMillis = this.timeout; + long maxTime = System.currentTimeMillis() + timeoutMillis; + int bytesWritten; + while (buffer.hasRemaining()) { + bytesWritten = channel.write(buffer); + if (bytesWritten == 0) { + if (System.currentTimeMillis() > maxTime) { + throw new SocketTimeoutException("Timed out writing to socket"); + } + try { + TimeUnit.NANOSECONDS.sleep(CHANNEL_FULL_WAIT_NANOS); + } catch (InterruptedException e) { + close(); + Thread.currentThread().interrupt(); // set the interrupt status + throw new ClosedByInterruptException(); // simulate an interrupted blocked write operation + } + } else { + maxTime = System.currentTimeMillis() + timeoutMillis; + } + } + } + + /** + * Closes the underlying SocketChannel + * @throws java.io.IOException + */ + @Override + public void close() throws IOException { + channel.close(); + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java new file mode 100644 index 0000000000..581048819a --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannel.java @@ -0,0 +1,602 @@ +/* + * 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.remote.io.socket.ssl; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketAddress; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.nio.channels.ClosedByInterruptException; +import java.nio.channels.SocketChannel; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLEngine; +import javax.net.ssl.SSLEngineResult; +import javax.net.ssl.SSLEngineResult.Status; +import javax.net.ssl.SSLHandshakeException; +import javax.net.ssl.SSLPeerUnverifiedException; +import javax.security.cert.CertificateExpiredException; +import javax.security.cert.CertificateNotYetValidException; +import javax.security.cert.X509Certificate; + +import org.apache.nifi.remote.exception.TransmissionDisabledException; +import org.apache.nifi.remote.io.socket.BufferStateManager; +import org.apache.nifi.remote.io.socket.BufferStateManager.Direction; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SSLSocketChannel implements Closeable { + + public static final int MAX_WRITE_SIZE = 65536; + + private static final Logger logger = LoggerFactory.getLogger(SSLSocketChannel.class); + private static final long BUFFER_FULL_EMPTY_WAIT_NANOS = TimeUnit.NANOSECONDS.convert(10, TimeUnit.MILLISECONDS); + + private final String hostname; + private final int port; + private final SSLEngine engine; + private final SocketAddress socketAddress; + + private BufferStateManager streamInManager; + private BufferStateManager streamOutManager; + private BufferStateManager appDataManager; + + private SocketChannel channel; + + private final byte[] oneByteBuffer = new byte[1]; + + private int timeoutMillis = 30000; + private volatile boolean connected = false; + private boolean handshaking = false; + private boolean closed = false; + private volatile boolean interrupted = false; + + public SSLSocketChannel(final SSLContext sslContext, final String hostname, final int port, final boolean client) throws IOException { + this.socketAddress = new InetSocketAddress(hostname, port); + this.channel = SocketChannel.open(); + this.hostname = hostname; + this.port = port; + this.engine = sslContext.createSSLEngine(); + this.engine.setUseClientMode(client); + engine.setNeedClientAuth(true); + + streamInManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize())); + streamOutManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize())); + appDataManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getApplicationBufferSize())); + } + + public SSLSocketChannel(final SSLContext sslContext, final SocketChannel socketChannel, final boolean client) throws IOException { + if (!socketChannel.isConnected()) { + throw new IllegalArgumentException("Cannot pass an un-connected SocketChannel"); + } + + this.channel = socketChannel; + + this.socketAddress = socketChannel.getRemoteAddress(); + final Socket socket = socketChannel.socket(); + this.hostname = socket.getInetAddress().getHostName(); + this.port = socket.getPort(); + + this.engine = sslContext.createSSLEngine(); + this.engine.setUseClientMode(client); + engine.setNeedClientAuth(true); + + streamInManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize())); + streamOutManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getPacketBufferSize())); + appDataManager = new BufferStateManager(ByteBuffer.allocate(engine.getSession().getApplicationBufferSize())); + } + + public void setTimeout(final int millis) { + this.timeoutMillis = millis; + } + + public int getTimeout() { + return timeoutMillis; + } + + public void connect() throws SSLHandshakeException, IOException { + try { + channel.configureBlocking(false); + if (!channel.isConnected()) { + final long startTime = System.currentTimeMillis(); + + if (!channel.connect(socketAddress)) { + while (!channel.finishConnect()) { + if (interrupted) { + throw new TransmissionDisabledException(); + } + if (System.currentTimeMillis() > startTime + timeoutMillis) { + throw new SocketTimeoutException("Timed out connecting to " + hostname + ":" + port); + } + + try { + Thread.sleep(50L); + } catch (final InterruptedException e) { + } + } + } + } + engine.beginHandshake(); + + performHandshake(); + logger.debug("{} Successfully completed SSL handshake", this); + + streamInManager.clear(); + streamOutManager.clear(); + appDataManager.clear(); + + connected = true; + } catch (final Exception e) { + logger.error("{} Failed to connect due to {}", this, e); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + closeQuietly(channel); + engine.closeInbound(); + engine.closeOutbound(); + throw e; + } + } + + public String getDn() throws CertificateExpiredException, CertificateNotYetValidException, SSLPeerUnverifiedException { + final X509Certificate[] certs = engine.getSession().getPeerCertificateChain(); + if (certs == null || certs.length == 0) { + throw new SSLPeerUnverifiedException("No certificates found"); + } + + final X509Certificate cert = certs[0]; + cert.checkValidity(); + return cert.getSubjectDN().getName().trim(); + } + + private void performHandshake() throws IOException { + // Generate handshake message + final byte[] emptyMessage = new byte[0]; + handshaking = true; + logger.debug("{} Performing Handshake", this); + + try { + while (true) { + switch (engine.getHandshakeStatus()) { + case FINISHED: + return; + case NEED_WRAP: { + final ByteBuffer appDataOut = ByteBuffer.wrap(emptyMessage); + + final ByteBuffer outboundBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + + final SSLEngineResult wrapHelloResult = engine.wrap(appDataOut, outboundBuffer); + if (wrapHelloResult.getStatus() == Status.BUFFER_OVERFLOW) { + streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + continue; + } + + if (wrapHelloResult.getStatus() != Status.OK) { + throw new SSLHandshakeException("Could not generate SSL Handshake information: SSLEngineResult: " + + wrapHelloResult.toString()); + } + + logger.trace("{} Handshake response after wrapping: {}", this, wrapHelloResult); + + final ByteBuffer readableStreamOut = streamOutManager.prepareForRead(1); + final int bytesToSend = readableStreamOut.remaining(); + writeFully(readableStreamOut); + logger.trace("{} Sent {} bytes of wrapped data for handshake", this, bytesToSend); + + streamOutManager.clear(); + } + continue; + case NEED_UNWRAP: { + final ByteBuffer readableDataIn = streamInManager.prepareForRead(0); + final ByteBuffer appData = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + + // Read handshake response from other side + logger.trace("{} Unwrapping: {} to {}", new Object[]{this, readableDataIn, appData}); + SSLEngineResult handshakeResponseResult = engine.unwrap(readableDataIn, appData); + logger.trace("{} Handshake response after unwrapping: {}", this, handshakeResponseResult); + + if (handshakeResponseResult.getStatus() == Status.BUFFER_UNDERFLOW) { + final ByteBuffer writableDataIn = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize()); + final int bytesRead = readData(writableDataIn); + if (bytesRead > 0) { + logger.trace("{} Read {} bytes for handshake", this, bytesRead); + } + + if (bytesRead < 0) { + throw new SSLHandshakeException("Reached End-of-File marker while performing handshake"); + } + } else if (handshakeResponseResult.getStatus() == Status.CLOSED) { + throw new IOException("Channel was closed by peer during handshake"); + } else { + streamInManager.compact(); + appDataManager.clear(); + } + } + break; + case NEED_TASK: + performTasks(); + continue; + case NOT_HANDSHAKING: + return; + } + } + } finally { + handshaking = false; + } + } + + private void performTasks() { + Runnable runnable; + while ((runnable = engine.getDelegatedTask()) != null) { + runnable.run(); + } + } + + private void closeQuietly(final Closeable closeable) { + try { + closeable.close(); + } catch (final Exception e) { + } + } + + private int readData(final ByteBuffer dest) throws IOException { + final long startTime = System.currentTimeMillis(); + + while (true) { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + if (dest.remaining() == 0) { + return 0; + } + + final int readCount = channel.read(dest); + + if (readCount == 0) { + if (System.currentTimeMillis() > startTime + timeoutMillis) { + throw new SocketTimeoutException("Timed out reading from socket connected to " + hostname + ":" + port); + } + try { + TimeUnit.NANOSECONDS.sleep(BUFFER_FULL_EMPTY_WAIT_NANOS); + } catch (InterruptedException e) { + close(); + Thread.currentThread().interrupt(); // set the interrupt status + throw new ClosedByInterruptException(); + } + + continue; + } + + logger.trace("{} Read {} bytes", this, readCount); + return readCount; + } + } + + private Status encryptAndWriteFully(final BufferStateManager src) throws IOException { + SSLEngineResult result = null; + + final ByteBuffer buff = src.prepareForRead(0); + final ByteBuffer outBuff = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + + logger.trace("{} Encrypting {} bytes", this, buff.remaining()); + while (buff.remaining() > 0) { + result = engine.wrap(buff, outBuff); + if (result.getStatus() == Status.OK) { + final ByteBuffer readableOutBuff = streamOutManager.prepareForRead(0); + writeFully(readableOutBuff); + streamOutManager.clear(); + } else { + return result.getStatus(); + } + } + + return result.getStatus(); + } + + private void writeFully(final ByteBuffer src) throws IOException { + long lastByteWrittenTime = System.currentTimeMillis(); + + int bytesWritten = 0; + while (src.hasRemaining()) { + if (interrupted) { + throw new TransmissionDisabledException(); + } + + final int written = channel.write(src); + bytesWritten += written; + final long now = System.currentTimeMillis(); + if (written > 0) { + lastByteWrittenTime = now; + } else { + if (now > lastByteWrittenTime + timeoutMillis) { + throw new SocketTimeoutException("Timed out writing to socket connected to " + hostname + ":" + port); + } + try { + TimeUnit.NANOSECONDS.sleep(BUFFER_FULL_EMPTY_WAIT_NANOS); + } catch (final InterruptedException e) { + close(); + Thread.currentThread().interrupt(); // set the interrupt status + throw new ClosedByInterruptException(); + } + } + } + + logger.trace("{} Wrote {} bytes", this, bytesWritten); + } + + public boolean isClosed() { + if (closed) { + return true; + } + // need to detect if peer has sent closure handshake...if so the answer is true + final ByteBuffer writableInBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize()); + int readCount = 0; + try { + readCount = channel.read(writableInBuffer); + } catch (IOException e) { + logger.error("{} Failed to readData due to {}", new Object[]{this, e}); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + readCount = -1; // treat the condition same as if End of Stream + } + if (readCount == 0) { + return false; + } + if (readCount > 0) { + logger.trace("{} Read {} bytes", this, readCount); + + final ByteBuffer streamInBuffer = streamInManager.prepareForRead(1); + final ByteBuffer appDataBuffer = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + try { + SSLEngineResult unwrapResponse = engine.unwrap(streamInBuffer, appDataBuffer); + logger.trace("{} When checking if closed, (handshake={}) Unwrap response: {}", new Object[]{this, handshaking, unwrapResponse}); + if (unwrapResponse.getStatus().equals(Status.CLOSED)) { + // Drain the incoming TCP buffer + final ByteBuffer discardBuffer = ByteBuffer.allocate(8192); + int bytesDiscarded = channel.read(discardBuffer); + while (bytesDiscarded > 0) { + discardBuffer.clear(); + bytesDiscarded = channel.read(discardBuffer); + } + engine.closeInbound(); + } else { + streamInManager.compact(); + return false; + } + } catch (IOException e) { + logger.error("{} Failed to check if closed due to {}. Closing channel.", new Object[]{this, e}); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + } + } + // either readCount is -1, indicating an end of stream, or the peer sent a closure handshake + // so go ahead and close down the channel + closeQuietly(channel.socket()); + closeQuietly(channel); + closed = true; + return true; + } + + @Override + public void close() throws IOException { + logger.debug("{} Closing Connection", this); + if (channel == null) { + return; + } + + if (closed) { + return; + } + + try { + engine.closeOutbound(); + + final byte[] emptyMessage = new byte[0]; + + final ByteBuffer appDataOut = ByteBuffer.wrap(emptyMessage); + final ByteBuffer outboundBuffer = streamOutManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + final SSLEngineResult handshakeResult = engine.wrap(appDataOut, outboundBuffer); + + if (handshakeResult.getStatus() != Status.CLOSED) { + throw new IOException("Invalid close state - will not send network data"); + } + + final ByteBuffer readableStreamOut = streamOutManager.prepareForRead(1); + writeFully(readableStreamOut); + } finally { + // Drain the incoming TCP buffer + final ByteBuffer discardBuffer = ByteBuffer.allocate(8192); + try { + int bytesDiscarded = channel.read(discardBuffer); + while (bytesDiscarded > 0) { + discardBuffer.clear(); + bytesDiscarded = channel.read(discardBuffer); + } + } catch (Exception e) { + } + + closeQuietly(channel.socket()); + closeQuietly(channel); + closed = true; + } + } + + private int copyFromAppDataBuffer(final byte[] buffer, final int offset, final int len) { + // If any data already exists in the application data buffer, copy it to the buffer. + final ByteBuffer appDataBuffer = appDataManager.prepareForRead(1); + + final int appDataRemaining = appDataBuffer.remaining(); + if (appDataRemaining > 0) { + final int bytesToCopy = Math.min(len, appDataBuffer.remaining()); + appDataBuffer.get(buffer, offset, bytesToCopy); + + final int bytesCopied = appDataRemaining - appDataBuffer.remaining(); + logger.trace("{} Copied {} ({}) bytes from unencrypted application buffer to user space", + new Object[]{this, bytesToCopy, bytesCopied}); + return bytesCopied; + } + return 0; + } + + public int available() throws IOException { + ByteBuffer appDataBuffer = appDataManager.prepareForRead(1); + ByteBuffer streamDataBuffer = streamInManager.prepareForRead(1); + final int buffered = appDataBuffer.remaining() + streamDataBuffer.remaining(); + if (buffered > 0) { + return buffered; + } + + final boolean wasAbleToRead = isDataAvailable(); + if (!wasAbleToRead) { + return 0; + } + + appDataBuffer = appDataManager.prepareForRead(1); + streamDataBuffer = streamInManager.prepareForRead(1); + return appDataBuffer.remaining() + streamDataBuffer.remaining(); + } + + public boolean isDataAvailable() throws IOException { + final ByteBuffer appDataBuffer = appDataManager.prepareForRead(1); + final ByteBuffer streamDataBuffer = streamInManager.prepareForRead(1); + + if (appDataBuffer.remaining() > 0 || streamDataBuffer.remaining() > 0) { + return true; + } + + final ByteBuffer writableBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize()); + final int bytesRead = channel.read(writableBuffer); + return (bytesRead > 0); + } + + public int read() throws IOException { + final int bytesRead = read(oneByteBuffer); + if (bytesRead == -1) { + return -1; + } + return oneByteBuffer[0] & 0xFF; + } + + public int read(final byte[] buffer) throws IOException { + return read(buffer, 0, buffer.length); + } + + public int read(final byte[] buffer, final int offset, final int len) throws IOException { + logger.debug("{} Reading up to {} bytes of data", this, len); + + if (!connected) { + connect(); + } + + int copied = copyFromAppDataBuffer(buffer, offset, len); + if (copied > 0) { + return copied; + } + + appDataManager.clear(); + + while (true) { + // prepare buffers and call unwrap + final ByteBuffer streamInBuffer = streamInManager.prepareForRead(1); + SSLEngineResult unwrapResponse = null; + final ByteBuffer appDataBuffer = appDataManager.prepareForWrite(engine.getSession().getApplicationBufferSize()); + unwrapResponse = engine.unwrap(streamInBuffer, appDataBuffer); + logger.trace("{} When reading data, (handshake={}) Unwrap response: {}", new Object[]{this, handshaking, unwrapResponse}); + + switch (unwrapResponse.getStatus()) { + case BUFFER_OVERFLOW: + throw new SSLHandshakeException("Buffer Overflow, which is not allowed to happen from an unwrap"); + case BUFFER_UNDERFLOW: { +// appDataManager.prepareForRead(engine.getSession().getApplicationBufferSize()); + + final ByteBuffer writableInBuffer = streamInManager.prepareForWrite(engine.getSession().getPacketBufferSize()); + final int bytesRead = readData(writableInBuffer); + if (bytesRead < 0) { + return -1; + } + + continue; + } + case CLOSED: + throw new IOException("Channel is closed"); + case OK: { + copied = copyFromAppDataBuffer(buffer, offset, len); + if (copied == 0) { + throw new IOException("Failed to decrypt data"); + } + streamInManager.compact(); + return copied; + } + } + } + } + + public void write(final int data) throws IOException { + write(new byte[]{(byte) data}, 0, 1); + } + + public void write(final byte[] data) throws IOException { + write(data, 0, data.length); + } + + public void write(final byte[] data, final int offset, final int len) throws IOException { + logger.debug("{} Writing {} bytes of data", this, len); + + if (!connected) { + connect(); + } + + int iterations = len / MAX_WRITE_SIZE; + if (len % MAX_WRITE_SIZE > 0) { + iterations++; + } + + for (int i = 0; i < iterations; i++) { + streamOutManager.clear(); + final int itrOffset = offset + i * MAX_WRITE_SIZE; + final int itrLen = Math.min(len - itrOffset, MAX_WRITE_SIZE); + final ByteBuffer byteBuffer = ByteBuffer.wrap(data, itrOffset, itrLen); + + final BufferStateManager buffMan = new BufferStateManager(byteBuffer, Direction.READ); + final Status status = encryptAndWriteFully(buffMan); + switch (status) { + case BUFFER_OVERFLOW: + streamOutManager.ensureSize(engine.getSession().getPacketBufferSize()); + appDataManager.ensureSize(engine.getSession().getApplicationBufferSize()); + continue; + case OK: + continue; + case CLOSED: + throw new IOException("Channel is closed"); + case BUFFER_UNDERFLOW: + throw new AssertionError("Got Buffer Underflow but should not have..."); + } + } + } + + public void interrupt() { + this.interrupted = true; + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java new file mode 100644 index 0000000000..154bd0895f --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelInputStream.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.remote.io.socket.ssl; + +import java.io.IOException; +import java.io.InputStream; + +public class SSLSocketChannelInputStream extends InputStream { + + private final SSLSocketChannel channel; + + public SSLSocketChannelInputStream(final SSLSocketChannel channel) { + this.channel = channel; + } + + @Override + public int read() throws IOException { + return channel.read(); + } + + @Override + public int read(final byte[] b) throws IOException { + return channel.read(b); + } + + @Override + public int read(final byte[] b, final int off, final int len) throws IOException { + return channel.read(b, off, len); + } + + /** + * Closes the underlying SSLSocketChannel, which will also close the + * OutputStream and connection + */ + @Override + public void close() throws IOException { + channel.close(); + } + + @Override + public int available() throws IOException { + return channel.available(); + } + + public boolean isDataAvailable() throws IOException { + return available() > 0; + } +} diff --git a/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java new file mode 100644 index 0000000000..ce4e4200fc --- /dev/null +++ b/commons/remote-communications-utils/src/main/java/org/apache/nifi/remote/io/socket/ssl/SSLSocketChannelOutputStream.java @@ -0,0 +1,53 @@ +/* + * 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.remote.io.socket.ssl; + +import java.io.IOException; +import java.io.OutputStream; + +public class SSLSocketChannelOutputStream extends OutputStream { + + private final SSLSocketChannel channel; + + public SSLSocketChannelOutputStream(final SSLSocketChannel channel) { + this.channel = channel; + } + + @Override + public void write(final int b) throws IOException { + channel.write(b); + } + + @Override + public void write(byte[] b) throws IOException { + channel.write(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + channel.write(b, off, len); + } + + /** + * Closes the underlying SSLSocketChannel, which also will close the + * InputStream and the connection + */ + @Override + public void close() throws IOException { + channel.close(); + } +} diff --git a/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java b/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java new file mode 100644 index 0000000000..bd30a966b6 --- /dev/null +++ b/commons/remote-communications-utils/src/test/java/org/apache/nifi/remote/io/TestCompressionInputOutputStreams.java @@ -0,0 +1,153 @@ +/* + * 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.remote.io; + +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; + +import org.apache.nifi.remote.io.CompressionInputStream; +import org.apache.nifi.remote.io.CompressionOutputStream; + +import org.junit.Test; + +public class TestCompressionInputOutputStreams { + + @Test + public void testSimple() throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final byte[] data = "Hello, World!".getBytes("UTF-8"); + + final CompressionOutputStream cos = new CompressionOutputStream(baos); + cos.write(data); + cos.flush(); + cos.close(); + + final byte[] compressedBytes = baos.toByteArray(); + final CompressionInputStream cis = new CompressionInputStream(new ByteArrayInputStream(compressedBytes)); + final byte[] decompressed = readFully(cis); + + assertTrue(Arrays.equals(data, decompressed)); + } + + @Test + public void testDataLargerThanBuffer() throws IOException { + final String str = "The quick brown fox jumps over the lazy dog\r\n\n\n\r"; + + final StringBuilder sb = new StringBuilder(); + for (int i = 0; i < 100; i++) { + sb.append(str); + } + final byte[] data = sb.toString().getBytes("UTF-8"); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final CompressionOutputStream cos = new CompressionOutputStream(baos, 8192); + cos.write(data); + cos.flush(); + cos.close(); + + final byte[] compressedBytes = baos.toByteArray(); + final CompressionInputStream cis = new CompressionInputStream(new ByteArrayInputStream(compressedBytes)); + final byte[] decompressed = readFully(cis); + + assertTrue(Arrays.equals(data, decompressed)); + } + + @Test + public void testDataLargerThanBufferWhileFlushing() throws IOException { + final String str = "The quick brown fox jumps over the lazy dog\r\n\n\n\r"; + final byte[] data = str.getBytes("UTF-8"); + + final StringBuilder sb = new StringBuilder(); + final byte[] data1024; + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final CompressionOutputStream cos = new CompressionOutputStream(baos, 8192); + for (int i = 0; i < 1024; i++) { + cos.write(data); + cos.flush(); + sb.append(str); + } + cos.close(); + data1024 = sb.toString().getBytes("UTF-8"); + + final byte[] compressedBytes = baos.toByteArray(); + final CompressionInputStream cis = new CompressionInputStream(new ByteArrayInputStream(compressedBytes)); + final byte[] decompressed = readFully(cis); + + assertTrue(Arrays.equals(data1024, decompressed)); + } + + @Test + public void testSendingMultipleFilesBackToBackOnSameStream() throws IOException { + final String str = "The quick brown fox jumps over the lazy dog\r\n\n\n\r"; + final byte[] data = str.getBytes("UTF-8"); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final CompressionOutputStream cos = new CompressionOutputStream(baos, 8192); + for (int i = 0; i < 512; i++) { + cos.write(data); + cos.flush(); + } + cos.close(); + + final CompressionOutputStream cos2 = new CompressionOutputStream(baos, 8192); + for (int i = 0; i < 512; i++) { + cos2.write(data); + cos2.flush(); + } + cos2.close(); + + final byte[] data512; + final StringBuilder sb = new StringBuilder(); + for (int i = 0; i < 512; i++) { + sb.append(str); + } + data512 = sb.toString().getBytes("UTF-8"); + + final byte[] compressedBytes = baos.toByteArray(); + final ByteArrayInputStream bais = new ByteArrayInputStream(compressedBytes); + + final CompressionInputStream cis = new CompressionInputStream(bais); + final byte[] decompressed = readFully(cis); + assertTrue(Arrays.equals(data512, decompressed)); + + final CompressionInputStream cis2 = new CompressionInputStream(bais); + final byte[] decompressed2 = readFully(cis2); + assertTrue(Arrays.equals(data512, decompressed2)); + } + + private byte[] readFully(final InputStream in) throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + + final byte[] buffer = new byte[65536]; + int len; + while ((len = in.read(buffer)) >= 0) { + baos.write(buffer, 0, len); + } + + return baos.toByteArray(); + } +} diff --git a/commons/search-utils/pom.xml b/commons/search-utils/pom.xml new file mode 100644 index 0000000000..569958fc7e --- /dev/null +++ b/commons/search-utils/pom.xml @@ -0,0 +1,32 @@ + + + 4.0.0 + + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + nifi-search-utils + 0.0.1-SNAPSHOT + jar + + search-utils + + + + diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.java new file mode 100644 index 0000000000..59b444a457 --- /dev/null +++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/Search.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.util.search; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Set; + +import org.apache.nifi.util.search.ahocorasick.SearchState; + +/** + * Defines an interface to search for content given a set of search terms. Any + * implementation of search must be thread safe. + * + * @author + * @param + */ +public interface Search { + + /** + * Establishes the dictionary of terms which will be searched in subsequent + * search calls. This can be called only once + * + * @param terms + */ + void initializeDictionary(Set> terms); + + /** + * Searches the given input stream for matches between the already specified + * dictionary and the contents scanned. + * + * @param haystack + * @param findAll if true will find all matches if false will find only the + * first match + * @return SearchState containing results Map might be empty which indicates + * no matches found but will not be null + * @throws IOException Thrown for any exceptions occurring while searching. + * @throws IllegalStateException if the dictionary has not yet been + * initialized + */ + SearchState search(InputStream haystack, boolean findAll) throws IOException; + +} diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java new file mode 100644 index 0000000000..62de964b9f --- /dev/null +++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/SearchTerm.java @@ -0,0 +1,141 @@ +/* + * 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.util.search; + +import java.nio.charset.Charset; +import java.util.Arrays; + +/** + * This is an immutable thread safe object representing a search term + * + * @author + * @param + */ +public class SearchTerm { + + private final byte[] bytes; + private final int hashCode; + private final T reference; + + /** + * Constructs a SearchTerm. Defensively copies the given byte array + * + * @param bytes + * @throws IllegalArgument exception if given bytes are null or 0 length + */ + public SearchTerm(final byte[] bytes) { + this(bytes, true, null); + } + + /** + * Constructs a search term. Optionally performs a defensive copy of the + * given byte array. If the caller indicates a defensive copy is not + * necessary then they must not change the given arrays state any longer + * + * @param bytes + * @param defensiveCopy + * @param reference + */ + public SearchTerm(final byte[] bytes, final boolean defensiveCopy, final T reference) { + if (bytes == null || bytes.length == 0) { + throw new IllegalArgumentException(); + } + if (defensiveCopy) { + this.bytes = Arrays.copyOf(bytes, bytes.length); + } else { + this.bytes = bytes; + } + this.hashCode = Arrays.hashCode(this.bytes); + this.reference = reference; + } + + public int get(final int index) { + return bytes[index] & 0xff; + } + + /** + * @return size in of search term in bytes + */ + public int size() { + return bytes.length; + } + + /** + * @return reference object for this given search term + */ + public T getReference() { + return reference; + } + + /** + * Determines if the given window starts with the same bytes as this term + * + * @param window Current window of bytes from the haystack being evaluated. + * @param windowLength The length of the window to consider + * @return true if this term starts with the same bytes of the given window + */ + public boolean startsWith(byte[] window, int windowLength) { + if (windowLength > window.length) { + throw new IndexOutOfBoundsException(); + } + if (bytes.length < windowLength) { + return false; + } + for (int i = 0; i < bytes.length && i < windowLength; i++) { + if (bytes[i] != window[i]) { + return false; + } + } + return true; + } + + /** + * @return a defensive copy of the internal byte structure + */ + public byte[] getBytes() { + return Arrays.copyOf(bytes, bytes.length); + } + + @Override + public int hashCode() { + return hashCode; + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final SearchTerm other = (SearchTerm) obj; + if (this.hashCode != other.hashCode) { + return false; + } + return Arrays.equals(this.bytes, other.bytes); + } + + @Override + public String toString() { + return new String(bytes); + } + + public String toString(final Charset charset) { + return new String(bytes, charset); + } +} diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java new file mode 100644 index 0000000000..3b8afaf39c --- /dev/null +++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/AhoCorasick.java @@ -0,0 +1,155 @@ +/* + * 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.util.search.ahocorasick; + +import java.io.IOException; +import java.io.InputStream; +import java.util.LinkedList; +import java.util.Queue; +import java.util.Set; + +import org.apache.nifi.util.search.Search; +import org.apache.nifi.util.search.SearchTerm; + +public class AhoCorasick implements Search { + + private Node root = null; + + /** + * Constructs a new search object. + * + * @throws IllegalArgumentException if given terms are null or empty + */ + public AhoCorasick() { + } + + @Override + public void initializeDictionary(final Set> terms) { + if (root != null) { + throw new IllegalStateException(); + } + root = new Node(); + if (terms == null || terms.isEmpty()) { + throw new IllegalArgumentException(); + } + for (final SearchTerm term : terms) { + int i = 0; + Node nextNode = root; + while (true) { + nextNode = addMatch(term, i, nextNode); + if (nextNode == null) { + break; //we're done + } + i++; + } + } + initialize(); + } + + private Node addMatch(final SearchTerm term, final int offset, final Node current) { + final int index = term.get(offset); + boolean atEnd = (offset == (term.size() - 1)); + if (current.getNeighbor(index) == null) { + if (atEnd) { + current.setNeighbor(new Node(term), index); + return null; + } + current.setNeighbor(new Node(), index); + } else if (atEnd) { + current.getNeighbor(index).setMatchingTerm(term); + return null; + } + return current.getNeighbor(index); + } + + private void initialize() { + //perform bgs to build failure links + final Queue queue = new LinkedList<>(); + queue.add(root); + root.setFailureNode(null); + while (!queue.isEmpty()) { + final Node current = queue.poll(); + for (int i = 0; i < 256; i++) { + final Node next = current.getNeighbor(i); + if (next != null) { + //traverse failure to get state + Node fail = current.getFailureNode(); + while ((fail != null) && fail.getNeighbor(i) == null) { + fail = fail.getFailureNode(); + } + if (fail != null) { + next.setFailureNode(fail.getNeighbor(i)); + } else { + next.setFailureNode(root); + } + queue.add(next); + } + } + } + } + + @Override + public SearchState search(final InputStream stream, final boolean findAll) throws IOException { + return search(stream, findAll, null); + } + + private SearchState search(final InputStream stream, final boolean findAll, final SearchState state) throws IOException { + if (root == null) { + throw new IllegalStateException(); + } + final SearchState currentState = (state == null) ? new SearchState(root) : state; + if (!findAll && currentState.foundMatch()) { + throw new IllegalStateException("A match has already been found yet we're being asked to keep searching"); + } + Node current = currentState.getCurrentNode(); + int currentChar; + while ((currentChar = stream.read()) >= 0) { + currentState.incrementBytesRead(1L); + Node next = current.getNeighbor(currentChar); + if (next == null) { + next = current.getFailureNode(); + while ((next != null) && next.getNeighbor(currentChar) == null) { + next = next.getFailureNode(); + } + if (next != null) { + next = next.getNeighbor(currentChar); + } else { + next = root; + } + } + if (next == null) { + throw new IllegalStateException("tree out of sync"); + } + //Accept condition + if (next.hasMatch()) { + currentState.addResult(next.getMatchingTerm()); + } + for (Node failNode = next.getFailureNode(); failNode != null; failNode = failNode.getFailureNode()) { + if (failNode.hasMatch()) { + currentState.addResult(failNode.getMatchingTerm()); + } + } + current = next; + if (currentState.foundMatch() && !findAll) { + break;//give up as soon as we have at least one match + } + } + currentState.setCurrentNode(current); + return currentState; + } + +} diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java new file mode 100644 index 0000000000..0ac325cf56 --- /dev/null +++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/Node.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.util.search.ahocorasick; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.util.search.SearchTerm; + +/** + * + * @author + */ +public class Node { + + private final Map neighborMap; + private Node failureNode; + private SearchTerm term; + + Node(final SearchTerm term) { + this(); + this.term = term; + } + + Node() { + neighborMap = new HashMap<>(); + term = null; + } + + void setFailureNode(final Node fail) { + failureNode = fail; + } + + public Node getFailureNode() { + return failureNode; + } + + public boolean hasMatch() { + return term != null; + } + + void setMatchingTerm(final SearchTerm term) { + this.term = term; + } + + public SearchTerm getMatchingTerm() { + return term; + } + + public Node getNeighbor(final int index) { + return neighborMap.get(index); + } + + void setNeighbor(final Node neighbor, final int index) { + neighborMap.put(index, neighbor); + } + +} diff --git a/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java new file mode 100644 index 0000000000..6d36ad0f4f --- /dev/null +++ b/commons/search-utils/src/main/java/org/apache/nifi/util/search/ahocorasick/SearchState.java @@ -0,0 +1,63 @@ +/* + * 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.util.search.ahocorasick; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.util.search.SearchTerm; + +public class SearchState { + + private Node currentNode; + private final Map, List> resultMap; + private long bytesRead; + + SearchState(final Node rootNode) { + resultMap = new HashMap<>(5); + currentNode = rootNode; + bytesRead = 0L; + } + + void incrementBytesRead(final long increment) { + bytesRead += increment; + } + + void setCurrentNode(final Node curr) { + currentNode = curr; + } + + public Node getCurrentNode() { + return currentNode; + } + + public Map, List> getResults() { + return new HashMap<>(resultMap); + } + + void addResult(final SearchTerm matchingTerm) { + final List indexes = (resultMap.containsKey(matchingTerm)) ? resultMap.get(matchingTerm) : new ArrayList(5); + indexes.add(bytesRead); + resultMap.put(matchingTerm, indexes); + } + + public boolean foundMatch() { + return !resultMap.isEmpty(); + } +} diff --git a/commons/wali/.gitignore b/commons/wali/.gitignore new file mode 100755 index 0000000000..19f2e002ce --- /dev/null +++ b/commons/wali/.gitignore @@ -0,0 +1,2 @@ +/target +/target diff --git a/commons/wali/pom.xml b/commons/wali/pom.xml new file mode 100644 index 0000000000..ce04973a1e --- /dev/null +++ b/commons/wali/pom.xml @@ -0,0 +1,42 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + wali + wali + + 3.0.0-SNAPSHOT + jar + WALI : Write-Ahead Log Implementation + + + + org.slf4j + slf4j-api + + + org.apache.nifi + nifi-stream-utils + 0.0.1-SNAPSHOT + + + diff --git a/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java b/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java new file mode 100644 index 0000000000..95cf4da06c --- /dev/null +++ b/commons/wali/src/main/java/org/wali/MinimalLockingWriteAheadLog.java @@ -0,0 +1,1008 @@ +/* + * 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.wali; + +import static java.util.Objects.requireNonNull; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.channels.FileChannel; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.regex.Pattern; + +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.io.BufferedOutputStream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + *

+ * This implementation provides as little Locking as possible in order to + * provide the highest throughput possible. However, this implementation is ONLY + * appropriate if it can be guaranteed that only a single thread will ever issue + * updates for a given Record at any one time. + *

+ * + * @param + */ +public final class MinimalLockingWriteAheadLog implements WriteAheadRepository { + + private final Path basePath; + private final Path partialPath; + private final Path snapshotPath; + + private final SerDe serde; + private final SyncListener syncListener; + private final FileChannel lockChannel; + private final AtomicLong transactionIdGenerator = new AtomicLong(0L); + + private final Partition[] partitions; + private final AtomicLong partitionIndex = new AtomicLong(0L); + private final ConcurrentMap recordMap = new ConcurrentHashMap<>(); + private final Map unmodifiableRecordMap = Collections.unmodifiableMap(recordMap); + private final Set externalLocations = new CopyOnWriteArraySet<>(); + + private final Set recoveredExternalLocations = new CopyOnWriteArraySet<>(); + + private final AtomicInteger numberBlackListedPartitions = new AtomicInteger(0); + + private static final Logger logger = LoggerFactory.getLogger(MinimalLockingWriteAheadLog.class); + + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); // required to update a partition + private final Lock writeLock = rwLock.writeLock(); // required for checkpoint + + private volatile boolean updated = false; + private volatile boolean recovered = false; + + public MinimalLockingWriteAheadLog(final Path path, final int partitionCount, final SerDe serde, final SyncListener syncListener) throws IOException { + this(new TreeSet<>(Collections.singleton(path)), partitionCount, serde, syncListener); + } + + /** + * + * @param paths a sorted set of Paths to use for the partitions/journals and + * the snapshot. The snapshot will always be written to the first path + * specified. + * + * @param partitionCount the number of partitions/journals to use. For best + * performance, this should be close to the number of threads that are + * expected to update the repository simultaneously + * + * @param serde + * @param syncListener + * @throws IOException + */ + @SuppressWarnings("unchecked") + public MinimalLockingWriteAheadLog(final SortedSet paths, final int partitionCount, final SerDe serde, final SyncListener syncListener) throws IOException { + this.syncListener = syncListener; + + requireNonNull(paths); + requireNonNull(serde); + + if (paths.isEmpty()) { + throw new IllegalArgumentException("Paths must be non-empty"); + } + + int existingPartitions = 0; + for (final Path path : paths) { + if (!Files.exists(path)) { + Files.createDirectories(path); + } + + final File file = path.toFile(); + if (!file.isDirectory()) { + throw new IOException("Path given [" + path + "] is not a directory"); + } + if (!file.canWrite()) { + throw new IOException("Path given [" + path + "] is not writable"); + } + if (!file.canRead()) { + throw new IOException("Path given [" + path + "] is not readable"); + } + if (!file.canExecute()) { + throw new IOException("Path given [" + path + "] is not executable"); + } + + final File[] children = file.listFiles(); + if (children != null) { + for (final File child : children) { + if (child.isDirectory() && child.getName().startsWith("partition-")) { + existingPartitions++; + } + } + + if (existingPartitions != 0 && existingPartitions != partitionCount) { + logger.warn("Constructing MinimalLockingWriteAheadLog with partitionCount={}, but the repository currently has " + + "{} partitions; ignoring argument and proceeding with {} partitions", + new Object[]{partitionCount, existingPartitions, existingPartitions}); + } + } + } + + this.basePath = paths.iterator().next(); + this.partialPath = basePath.resolve("snapshot.partial"); + this.snapshotPath = basePath.resolve("snapshot"); + this.serde = serde; + + final Path lockPath = basePath.resolve("wali.lock"); + lockChannel = new FileOutputStream(lockPath.toFile()).getChannel(); + lockChannel.lock(); + + partitions = new Partition[partitionCount]; + + Iterator pathIterator = paths.iterator(); + for (int i = 0; i < partitionCount; i++) { + // If we're out of paths, create a new iterator to start over. + if (!pathIterator.hasNext()) { + pathIterator = paths.iterator(); + } + + final Path partitionBasePath = pathIterator.next(); + + partitions[i] = new Partition<>(partitionBasePath.resolve("partition-" + i), serde, i, getVersion()); + } + } + + @Override + public int update(final Collection records, final boolean forceSync) throws IOException { + if (!recovered) { + throw new IllegalStateException("Cannot update repository until record recovery has been performed"); + } + + if (records.isEmpty()) { + return -1; + } + + updated = true; + readLock.lock(); + try { + while (true) { + final int numBlackListed = numberBlackListedPartitions.get(); + if (numBlackListed >= partitions.length) { + throw new IOException("All Partitions have been blacklisted due to failures when attempting to update. If the Write-Ahead Log is able to perform a checkpoint, this issue may resolve itself. Otherwise, manual intervention will be required."); + } + + final long partitionIdx = partitionIndex.getAndIncrement(); + final int resolvedIdx = (int) (partitionIdx % partitions.length); + final Partition partition = partitions[resolvedIdx]; + if (partition.tryClaim()) { + try { + final long transactionId = transactionIdGenerator.getAndIncrement(); + if (logger.isTraceEnabled()) { + for (final T record : records) { + logger.trace("Partition {} performing Transaction {}: {}", new Object[]{partition, transactionId, record}); + } + } + + try { + partition.update(records, transactionId, unmodifiableRecordMap, forceSync); + } catch (final Exception e) { + partition.blackList(); + numberBlackListedPartitions.incrementAndGet(); + throw e; + } + + if (forceSync && syncListener != null) { + syncListener.onSync(resolvedIdx); + } + } finally { + partition.releaseClaim(); + } + + for (final T record : records) { + final UpdateType updateType = serde.getUpdateType(record); + final Object recordIdentifier = serde.getRecordIdentifier(record); + + if (updateType == UpdateType.DELETE) { + recordMap.remove(recordIdentifier); + } else if (updateType == UpdateType.SWAP_OUT) { + final String newLocation = serde.getLocation(record); + if (newLocation == null) { + logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_OUT but no indicator of where the Record is to be Swapped Out to; these records may be lost when the repository is restored!"); + } else { + recordMap.remove(recordIdentifier); + this.externalLocations.add(newLocation); + } + } else if (updateType == UpdateType.SWAP_IN) { + final String newLocation = serde.getLocation(record); + if (newLocation == null) { + logger.error("Received Record (ID=" + recordIdentifier + ") with UpdateType of SWAP_IN but no indicator of where the Record is to be Swapped In from; these records may be duplicated when the repository is restored!"); + } else { + externalLocations.remove(newLocation); + } + recordMap.put(recordIdentifier, record); + } else { + recordMap.put(recordIdentifier, record); + } + } + + return resolvedIdx; + } + } + } finally { + readLock.unlock(); + } + } + + @Override + public Collection recoverRecords() throws IOException { + if (updated) { + throw new IllegalStateException("Cannot recover records after updating the repository; must call recoverRecords first"); + } + + final long recoverStart = System.nanoTime(); + writeLock.lock(); + try { + Long maxTransactionId = recoverFromSnapshot(recordMap); + recoverFromEdits(recordMap, maxTransactionId); + + for (final Partition partition : partitions) { + final long transId = partition.getMaxRecoveredTransactionId(); + if (maxTransactionId == null || transId > maxTransactionId) { + maxTransactionId = transId; + } + } + + this.transactionIdGenerator.set(maxTransactionId + 1); + this.externalLocations.addAll(recoveredExternalLocations); + logger.info("{} finished recovering records. Performing Checkpoint to ensure proper state of Partitions before updates", this); + } finally { + writeLock.unlock(); + } + final long recoverNanos = System.nanoTime() - recoverStart; + final long recoveryMillis = TimeUnit.MILLISECONDS.convert(recoverNanos, TimeUnit.NANOSECONDS); + logger.info("Successfully recovered {} records in {} milliseconds", recordMap.size(), recoveryMillis); + checkpoint(); + + recovered = true; + return recordMap.values(); + } + + @Override + public Set getRecoveredSwapLocations() throws IOException { + return recoveredExternalLocations; + } + + private Long recoverFromSnapshot(final Map recordMap) throws IOException { + final boolean partialExists = Files.exists(partialPath); + final boolean snapshotExists = Files.exists(snapshotPath); + + if (!partialExists && !snapshotExists) { + return null; + } + + if (partialExists && snapshotExists) { + // both files exist -- assume we failed while checkpointing. Delete + // the partial file + Files.delete(partialPath); + } else if (partialExists) { + // partial exists but snapshot does not -- we must have completed + // creating the partial, deleted the snapshot + // but crashed before renaming the partial to the snapshot. Just + // rename partial to snapshot + Files.move(partialPath, snapshotPath); + } + + if (Files.size(snapshotPath) == 0) { + logger.warn("{} Found 0-byte Snapshot file; skipping Snapshot file in recovery", this); + return null; + } + + // at this point, we know the snapshotPath exists because if it didn't, then we either returned null + // or we renamed partialPath to snapshotPath. So just Recover from snapshotPath. + try (final DataInputStream dataIn = new DataInputStream(new BufferedInputStream(Files.newInputStream(snapshotPath, StandardOpenOption.READ)))) { + final String waliImplementationClass = dataIn.readUTF(); + final int waliImplementationVersion = dataIn.readInt(); + + if (!waliImplementationClass.equals(MinimalLockingWriteAheadLog.class.getName())) { + throw new IOException("Write-Ahead Log located at " + snapshotPath + " was written using the " + waliImplementationClass + " class; cannot restore using " + getClass().getName()); + } + + if (waliImplementationVersion > getVersion()) { + throw new IOException("Write-Ahead Log located at " + snapshotPath + " was written using version " + waliImplementationVersion + " of the " + waliImplementationClass + " class; cannot restore using Version " + getVersion()); + } + + dataIn.readUTF(); // ignore serde class name for now + final int serdeVersion = dataIn.readInt(); + final long maxTransactionId = dataIn.readLong(); + final int numRecords = dataIn.readInt(); + + for (int i = 0; i < numRecords; i++) { + final T record = serde.deserializeRecord(dataIn, serdeVersion); + if (record == null) { + throw new EOFException(); + } + + final UpdateType updateType = serde.getUpdateType(record); + if (updateType == UpdateType.DELETE) { + logger.warn("While recovering from snapshot, found record with type 'DELETE'; this record will not be restored"); + continue; + } + + logger.trace("Recovered from snapshot: {}", record); + recordMap.put(serde.getRecordIdentifier(record), record); + } + + final int numSwapRecords = dataIn.readInt(); + final Set swapLocations = new HashSet<>(); + for (int i = 0; i < numSwapRecords; i++) { + swapLocations.add(dataIn.readUTF()); + } + this.recoveredExternalLocations.addAll(swapLocations); + + logger.debug("{} restored {} Records and {} Swap Files from Snapshot, ending with Transaction ID {}", new Object[]{this, numRecords, recoveredExternalLocations.size(), maxTransactionId}); + return maxTransactionId; + } + } + + /** + * Recovers records from the edit logs via the Partitions. Returns a boolean + * if recovery of a Partition requires the Write-Ahead Log be checkpointed + * before modification. + * + * @param modifiableRecordMap + * @param maxTransactionIdRestored + * @return + * @throws IOException + */ + private void recoverFromEdits(final Map modifiableRecordMap, final Long maxTransactionIdRestored) throws IOException { + final Map updateMap = new HashMap<>(); + final Map unmodifiableRecordMap = Collections.unmodifiableMap(modifiableRecordMap); + final Map ignorableMap = new HashMap<>(); + final Set ignorableSwapLocations = new HashSet<>(); + + // populate a map of the next transaction id for each partition to the + // partition that has that next transaction id. + final SortedMap> transactionMap = new TreeMap<>(); + for (final Partition partition : partitions) { + Long transactionId; + boolean keepTransaction; + do { + transactionId = partition.getNextRecoverableTransactionId(); + + keepTransaction = transactionId == null || maxTransactionIdRestored == null || transactionId > maxTransactionIdRestored; + if (keepTransaction && transactionId != null) { + // map this transaction id to its partition so that we can + // start restoring transactions from this partition, + // starting at 'transactionId' + transactionMap.put(transactionId, partition); + } else if (transactionId != null) { + // skip the next transaction, because our snapshot already + // contained this transaction. + try { + partition.recoverNextTransaction(ignorableMap, updateMap, ignorableSwapLocations); + } catch (final EOFException e) { + logger.error("{} unexpectedly reached End of File while reading from {} for Transaction {}; assuming crash and ignoring this transaction.", + new Object[]{this, partition, transactionId}); + } + } + } while (!keepTransaction); + } + + while (!transactionMap.isEmpty()) { + final Map.Entry> firstEntry = transactionMap.entrySet().iterator().next(); + final Long firstTransactionId = firstEntry.getKey(); + final Partition nextPartition = firstEntry.getValue(); + + try { + updateMap.clear(); + final Set idsRemoved = nextPartition.recoverNextTransaction(unmodifiableRecordMap, updateMap, recoveredExternalLocations); + modifiableRecordMap.putAll(updateMap); + for (final Object id : idsRemoved) { + modifiableRecordMap.remove(id); + } + } catch (final EOFException e) { + logger.error("{} unexpectedly reached End-of-File when reading from {} for Transaction ID {}; assuming crash and ignoring this transaction", + new Object[]{this, nextPartition, firstTransactionId}); + } + + transactionMap.remove(firstTransactionId); + + Long subsequentTransactionId = null; + try { + subsequentTransactionId = nextPartition.getNextRecoverableTransactionId(); + } catch (final IOException e) { + logger.error("{} unexpectedly found End-of-File when reading from {} for Transaction ID {}; assuming crash and ignoring this transaction", + new Object[]{this, nextPartition, firstTransactionId}); + } + + if (subsequentTransactionId != null) { + transactionMap.put(subsequentTransactionId, nextPartition); + } + } + + for (final Partition partition : partitions) { + partition.endRecovery(); + } + } + + @Override + public synchronized int checkpoint() throws IOException { + final Set records; + final Set swapLocations; + final long maxTransactionId; + + final long startNanos = System.nanoTime(); + + FileOutputStream fileOut = null; + DataOutputStream dataOut = null; + + long stopTheWorldNanos = -1L; + long stopTheWorldStart = -1L; + try { + writeLock.lock(); + try { + stopTheWorldStart = System.nanoTime(); + // stop the world while we make a copy of the records that must + // be checkpointed and rollover the partitions. + // We copy the records because serializing them is potentially + // very expensive, especially when we have hundreds + // of thousands or even millions of them. We don't want to + // prevent WALI from being used during this time. + + // So the design is to copy all of the records, determine the + // last transaction ID that the records represent, + // and roll over the partitions to new write-ahead logs. + // Then, outside of the write lock, we will serialize the data + // to disk, and then remove the old Partition data. + records = new HashSet<>(recordMap.values()); + maxTransactionId = transactionIdGenerator.get() - 1; + + swapLocations = new HashSet<>(externalLocations); + for (final Partition partition : partitions) { + partition.rollover(); + } + + // notify global sync with the write lock held. We do this because we don't want the repository to get updated + // while the listener is performing its necessary tasks + if (syncListener != null) { + syncListener.onGlobalSync(); + } + } finally { + writeLock.unlock(); + } + + stopTheWorldNanos = System.nanoTime() - stopTheWorldStart; + + // perform checkpoint, writing to .partial file + fileOut = new FileOutputStream(partialPath.toFile()); + dataOut = new DataOutputStream(fileOut); + dataOut.writeUTF(MinimalLockingWriteAheadLog.class.getName()); + dataOut.writeInt(getVersion()); + dataOut.writeUTF(serde.getClass().getName()); + dataOut.writeInt(serde.getVersion()); + dataOut.writeLong(maxTransactionId); + dataOut.writeInt(records.size()); + + for (final T record : records) { + logger.trace("Checkpointing {}", record); + serde.serializeRecord(record, dataOut); + } + + dataOut.writeInt(swapLocations.size()); + for (final String swapLocation : swapLocations) { + dataOut.writeUTF(swapLocation); + } + } finally { + if (dataOut != null) { + try { + dataOut.flush(); + fileOut.getFD().sync(); + dataOut.close(); + } catch (final IOException e) { + logger.warn("Failed to close Data Stream due to {}", e.toString(), e); + } + } + } + + // delete the snapshot, if it exists, and rename the .partial to + // snapshot + Files.deleteIfExists(snapshotPath); + Files.move(partialPath, snapshotPath); + + // clear all of the edit logs + final long partitionStart = System.nanoTime(); + for (final Partition partition : partitions) { + // we can call clearOld without claiming the partition because it + // does not change the partition's state + // and the only member variable it touches cannot be modified, other + // than when #rollover() is called. + // And since this method is the only one that calls #rollover() and + // this method is synchronized, + // the value of that member variable will not change. And it's + // volatile, so we will get the correct value. + partition.clearOld(); + } + final long partitionEnd = System.nanoTime(); + numberBlackListedPartitions.set(0); + + final long endNanos = System.nanoTime(); + final long millis = TimeUnit.MILLISECONDS.convert(endNanos - startNanos, TimeUnit.NANOSECONDS); + final long partitionMillis = TimeUnit.MILLISECONDS.convert(partitionEnd - partitionStart, TimeUnit.NANOSECONDS); + final long stopTheWorldMillis = TimeUnit.NANOSECONDS.toMillis(stopTheWorldNanos); + + logger.info("{} checkpointed with {} Records and {} Swap Files in {} milliseconds (Stop-the-world time = {} milliseconds, Clear Edit Logs time = {} millis), max Transaction ID {}", + new Object[]{this, records.size(), swapLocations.size(), millis, stopTheWorldMillis, partitionMillis, maxTransactionId}); + + return records.size(); + } + + @Override + public void shutdown() throws IOException { + writeLock.lock(); + try { + for (final Partition partition : partitions) { + partition.close(); + } + } finally { + writeLock.unlock(); + lockChannel.close(); + } + } + + public int getVersion() { + return 1; + } + + /** + * Represents a partition of this repository, which maps directly to a + * .journal file. + * + * All methods with the exceptions of {@link #claim()}, {@link #tryClaim()}, + * and {@link #releaseClaim()} in this Partition MUST be called while + * holding the claim (via {@link #claim} or {@link #tryClaim()). + * + * @param + */ + private static class Partition { + + public static final String JOURNAL_EXTENSION = ".journal"; + private static final Pattern JOURNAL_FILENAME_PATTERN = Pattern.compile("\\d+\\.journal"); + + private final SerDe serde; + + private final Path editDirectory; + private final int writeAheadLogVersion; + + private final Lock lock = new ReentrantLock(); + private DataOutputStream dataOut = null; + private FileOutputStream fileOut = null; + private boolean blackListed = false; + private boolean closed = false; + private DataInputStream recoveryIn; + private int recoveryVersion; + private String currentJournalFilename = ""; + + private static final byte TRANSACTION_CONTINUE = 1; + private static final byte TRANSACTION_COMMIT = 2; + + private final String description; + private final AtomicLong maxTransactionId = new AtomicLong(-1L); + private final Logger logger = LoggerFactory.getLogger(MinimalLockingWriteAheadLog.class); + + private final Queue recoveryFiles; + + public Partition(final Path path, final SerDe serde, final int partitionIndex, final int writeAheadLogVersion) throws IOException { + this.editDirectory = path; + this.serde = serde; + + final File file = path.toFile(); + if (!file.exists() && !file.mkdirs()) { + throw new IOException("Could not create directory " + file.getAbsolutePath()); + } + + this.recoveryFiles = new LinkedBlockingQueue<>(); + for (final Path recoveryPath : getRecoveryPaths()) { + recoveryFiles.add(recoveryPath); + } + + this.description = "Partition-" + partitionIndex; + this.writeAheadLogVersion = writeAheadLogVersion; + } + + public boolean tryClaim() { + final boolean obtainedLock = lock.tryLock(); + if (!obtainedLock) { + return false; + } + + // Check if the partition is blacklisted. If so, unlock it and return false. Otherwise, + // leave it locked and return true, so that the caller will need to unlock. + if (blackListed) { + lock.unlock(); + return false; + } + + return true; + } + + public void releaseClaim() { + lock.unlock(); + } + + public void close() { + final DataOutputStream out = dataOut; + if (out != null) { + try { + out.close(); + } catch (final Exception e) { + + } + } + + this.closed = true; + this.dataOut = null; + } + + public void blackList() { + lock.lock(); + try { + blackListed = true; + } finally { + lock.unlock(); + } + logger.debug("Blacklisted {}", this); + } + + /** + * Closes resources pointing to the current journal and begins writing + * to a new one + * + * @throws IOException + */ + public void rollover() throws IOException { + lock.lock(); + try { + final DataOutputStream out = dataOut; + if (out != null) { + out.close(); + } + + final Path editPath = getNewEditPath(); + final FileOutputStream fos = new FileOutputStream(editPath.toFile()); + final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos)); + outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName()); + outStream.writeInt(writeAheadLogVersion); + outStream.writeUTF(serde.getClass().getName()); + outStream.writeInt(serde.getVersion()); + outStream.flush(); + dataOut = outStream; + fileOut = fos; + + currentJournalFilename = editPath.toFile().getName(); + + blackListed = false; + } finally { + lock.unlock(); + } + } + + private long getJournalIndex(final File file) { + final String filename = file.getName(); + final int dotIndex = filename.indexOf("."); + final String number = filename.substring(0, dotIndex); + return Long.parseLong(number); + } + + private Path getNewEditPath() { + final List recoveryPaths = getRecoveryPaths(); + final long newIndex; + if (recoveryPaths == null || recoveryPaths.isEmpty()) { + newIndex = 1; + } else { + final long lastFileIndex = getJournalIndex(recoveryPaths.get(recoveryPaths.size() - 1).toFile()); + newIndex = lastFileIndex + 1; + } + + return editDirectory.resolve(newIndex + JOURNAL_EXTENSION); + } + + private List getRecoveryPaths() { + final List paths = new ArrayList<>(); + + final File directory = editDirectory.toFile(); + final File[] partitionFiles = directory.listFiles(); + if (partitionFiles == null) { + return paths; + } + + for (final File file : partitionFiles) { + // if file is a journal file but no data has yet been persisted, it may + // very well be a 0-byte file (the journal is not SYNC'ed to disk after + // a header is written out, so it may be lost). In this case, the journal + // is empty, so we can just skip it. + if (file.isDirectory() || file.length() == 0L) { + continue; + } + + if (!JOURNAL_FILENAME_PATTERN.matcher(file.getName()).matches()) { + continue; + } + + if (isJournalFile(file)) { + paths.add(file.toPath()); + } else { + logger.warn("Found file {}, but could not access it, or it was not in the expected format; will ignore this file", file.getAbsolutePath()); + } + } + + // Sort journal files by the numeric portion of the filename + Collections.sort(paths, new Comparator() { + @Override + public int compare(final Path o1, final Path o2) { + if (o1 == null && o2 == null) { + return 0; + } + if (o1 == null) { + return 1; + } + if (o2 == null) { + return -1; + } + + final long index1 = getJournalIndex(o1.toFile()); + final long index2 = getJournalIndex(o2.toFile()); + return Long.compare(index1, index2); + } + }); + + return paths; + } + + void clearOld() { + final List oldRecoveryFiles = getRecoveryPaths(); + + for (final Path path : oldRecoveryFiles) { + final File file = path.toFile(); + if (file.getName().equals(currentJournalFilename)) { + continue; + } + if (file.exists()) { + file.delete(); + } + } + } + + private boolean isJournalFile(final File file) { + final String expectedStartsWith = MinimalLockingWriteAheadLog.class.getName(); + try { + try (final FileInputStream fis = new FileInputStream(file); + final InputStream bufferedIn = new BufferedInputStream(fis); + final DataInputStream in = new DataInputStream(bufferedIn)) { + final String waliImplClassName = in.readUTF(); + if (!expectedStartsWith.equals(waliImplClassName)) { + return false; + } + } + } catch (final IOException e) { + return false; + } + + return true; + } + + public void update(final Collection records, final long transactionId, final Map recordMap, final boolean forceSync) throws IOException { + if (this.closed) { + throw new IllegalStateException("Partition is closed"); + } + + final DataOutputStream out = dataOut; + out.writeLong(transactionId); + + final int numEditsToSerialize = records.size(); + int editsSerialized = 0; + for (final S record : records) { + final Object recordId = serde.getRecordIdentifier(record); + final S previousVersion = recordMap.get(recordId); + + serde.serializeEdit(previousVersion, record, out); + if (++editsSerialized < numEditsToSerialize) { + out.write(TRANSACTION_CONTINUE); + } else { + out.write(TRANSACTION_COMMIT); + } + } + + out.flush(); + + if (forceSync) { + fileOut.getFD().sync(); + } + } + + private DataInputStream createDataInputStream(final Path path) throws IOException { + return new DataInputStream(new BufferedInputStream(Files.newInputStream(path))); + } + + private DataInputStream getRecoveryStream() throws IOException { + if (recoveryIn != null && hasMoreData(recoveryIn)) { + return recoveryIn; + } + + while (true) { + final Path nextRecoveryPath = recoveryFiles.poll(); + if (nextRecoveryPath == null) { + return null; + } + + recoveryIn = createDataInputStream(nextRecoveryPath); + if (hasMoreData(recoveryIn)) { + final String waliImplementationClass = recoveryIn.readUTF(); + if (!MinimalLockingWriteAheadLog.class.getName().equals(waliImplementationClass)) { + continue; + } + + final long waliVersion = recoveryIn.readInt(); + if (waliVersion > writeAheadLogVersion) { + throw new IOException("Cannot recovery from file " + nextRecoveryPath + " because it was written using WALI version " + waliVersion + ", but the version used to restore it is only " + writeAheadLogVersion); + } + + @SuppressWarnings("unused") + final String serdeClassName = recoveryIn.readUTF(); + this.recoveryVersion = recoveryIn.readInt(); + + break; + } + } + + return recoveryIn; + } + + public Long getNextRecoverableTransactionId() throws IOException { + while (true) { + DataInputStream recoveryStream = getRecoveryStream(); + if (recoveryStream == null) { + return null; + } + + final long transactionId; + try { + transactionId = recoveryIn.readLong(); + } catch (final EOFException e) { + continue; + } + + this.maxTransactionId.set(transactionId); + return transactionId; + } + } + + private boolean hasMoreData(final InputStream in) throws IOException { + in.mark(1); + final int nextByte = in.read(); + in.reset(); + return nextByte >= 0; + } + + public void endRecovery() throws IOException { + if (recoveryIn != null) { + recoveryIn.close(); + } + + final Path nextRecoveryPath = this.recoveryFiles.poll(); + if (nextRecoveryPath != null) { + throw new IllegalStateException("Signaled to end recovery, but there are more recovery files for Partition in directory " + editDirectory); + } + + final Path newEditPath = getNewEditPath(); + + final FileOutputStream fos = new FileOutputStream(newEditPath.toFile()); + final DataOutputStream outStream = new DataOutputStream(new BufferedOutputStream(fos)); + outStream.writeUTF(MinimalLockingWriteAheadLog.class.getName()); + outStream.writeInt(writeAheadLogVersion); + outStream.writeUTF(serde.getClass().getName()); + outStream.writeInt(serde.getVersion()); + outStream.flush(); + dataOut = outStream; + fileOut = fos; + } + + public Set recoverNextTransaction(final Map currentRecordMap, final Map updatedRecordMap, final Set swapLocations) throws IOException { + final Set idsRemoved = new HashSet<>(); + + int transactionFlag; + do { + final S record = serde.deserializeEdit(recoveryIn, currentRecordMap, recoveryVersion); + if (logger.isTraceEnabled()) { + logger.trace("{} Recovering Transaction {}: {}", new Object[]{this, maxTransactionId.get(), record}); + } + + final Object recordId = serde.getRecordIdentifier(record); + final UpdateType updateType = serde.getUpdateType(record); + if (updateType == UpdateType.DELETE) { + updatedRecordMap.remove(recordId); + idsRemoved.add(recordId); + } else if (updateType == UpdateType.SWAP_IN) { + final String location = serde.getLocation(record); + if (location == null) { + logger.error("Recovered SWAP_IN record from edit log, but it did not contain a Location; skipping record"); + } else { + swapLocations.remove(location); + updatedRecordMap.put(recordId, record); + idsRemoved.remove(recordId); + } + } else if (updateType == UpdateType.SWAP_OUT) { + final String location = serde.getLocation(record); + if (location == null) { + logger.error("Recovered SWAP_OUT record from edit log, but it did not contain a Location; skipping record"); + } else { + swapLocations.add(location); + updatedRecordMap.remove(recordId); + idsRemoved.add(recordId); + } + } else { + updatedRecordMap.put(recordId, record); + idsRemoved.remove(recordId); + } + + transactionFlag = recoveryIn.read(); + } while (transactionFlag != TRANSACTION_COMMIT); + + return idsRemoved; + } + + /** + * Must be called after recovery has finished + * + * @return + */ + public long getMaxRecoveredTransactionId() { + return maxTransactionId.get(); + } + + @Override + public String toString() { + return description; + } + } +} diff --git a/commons/wali/src/main/java/org/wali/SerDe.java b/commons/wali/src/main/java/org/wali/SerDe.java new file mode 100644 index 0000000000..bbc7efb8ae --- /dev/null +++ b/commons/wali/src/main/java/org/wali/SerDe.java @@ -0,0 +1,128 @@ +/* + * 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.wali; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.Map; + +/** + * A mechanism for Serializing and De-Serializing a Record of a given Type + * + * @param the type of record that is to be Serialized and De-Serialized by + * this object + */ +public interface SerDe { + + /** + *

+ * Serializes an Edit Record to the log via the given + * {@link DataOutputStream}. + *

+ * + * @param previousRecordState + * @param newRecordState + * @param out + * @throws IOException + */ + void serializeEdit(T previousRecordState, T newRecordState, DataOutputStream out) throws IOException; + + /** + *

+ * Serializes a Record in a form suitable for a Snapshot via the given + * {@link DataOutputStream}. + *

+ * + * @param record + * @param out + * @throws IOException + */ + void serializeRecord(T record, DataOutputStream out) throws IOException; + + /** + *

+ * Reads an Edit Record from the given {@link DataInputStream} and merges + * that edit with the current version of the record, returning the new, + * merged version. If the Edit Record indicates that the entity was deleted, + * must return a Record with an UpdateType of {@link UpdateType#DELETE}. + * This method must never return null. + *

+ * + * @param in + * @param currentRecordStates an unmodifiable map of Record ID's to the + * current state of that record + * @param version the version of the SerDe that was used to serialize the + * edit record + * @return + * @throws IOException + */ + T deserializeEdit(DataInputStream in, Map currentRecordStates, int version) throws IOException; + + /** + *

+ * Reads a Record from the given {@link DataInputStream} and returns this + * record. If no data is available, returns null. + *

+ * + * @param in + * @param version the version of the SerDe that was used to serialize the + * record + * @return + * @throws IOException + */ + T deserializeRecord(DataInputStream in, int version) throws IOException; + + /** + * Returns the unique ID for the given record + * + * @param record + * @return + */ + Object getRecordIdentifier(T record); + + /** + * Returns the UpdateType for the given record + * + * @param record + * @return + */ + UpdateType getUpdateType(T record); + + /** + * Returns the external location of the given record; this is used when a + * record is moved away from WALI or is being re-introduced to WALI. For + * example, WALI can be updated with a record of type + * {@link UpdateType#SWAP_OUT} that indicates a Location of + * file://tmp/external1 and can then be re-introduced to WALI by updating + * WALI with a record of type {@link UpdateType#CREATE} that indicates a + * Location of file://tmp/external1 + * + * @param record + * @return + */ + String getLocation(T record); + + /** + * Returns the version that this SerDe will use when writing. This used used + * when serializing/deserializing the edit logs so that if the version + * changes, we are still able to deserialize old versions + * + * @return + */ + int getVersion(); +} diff --git a/commons/wali/src/main/java/org/wali/SyncListener.java b/commons/wali/src/main/java/org/wali/SyncListener.java new file mode 100644 index 0000000000..ffb11cadfc --- /dev/null +++ b/commons/wali/src/main/java/org/wali/SyncListener.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.wali; + +/** + *

+ * Provides a callback mechanism by which applicable listeners can be notified + * when a WriteAheadRepository is synched (via the + * {@link WriteAheadRepository#sync()} method) or one of its partitions is + * synched via + * {@link WriteAheadRepository#update(java.util.Collection, boolean)} with a + * value of true for the second argument. + *

+ * + *

+ * It is not required that an implementation of {@link WriteAheadRepository} + * support this interface. Those that do generally will require that the + * listener be injected via the constructor. + *

+ * + *

+ * All implementations of this interface must be thread-safe. + *

+ * + *

+ * The {@link #onSync(int)} method will always be called while the associated + * partition is locked. The {@link #onGlobalSync()} will always be called while + * the entire repository is locked. + *

+ * + */ +public interface SyncListener { + + /** + * This method is called whenever a specific partition is synched via the + * {@link WriteAheadRepository#update(java.util.Collection, boolean)} method + * + * @param partitionIndex the index of the partition that was synched + */ + void onSync(int partitionIndex); + + /** + * This method is called whenever the entire + * WriteAheadRepository is synched via the + * {@link WriteAheadRepository#sync()} method. + */ + void onGlobalSync(); +} diff --git a/commons/wali/src/main/java/org/wali/UpdateType.java b/commons/wali/src/main/java/org/wali/UpdateType.java new file mode 100644 index 0000000000..1b039f82fc --- /dev/null +++ b/commons/wali/src/main/java/org/wali/UpdateType.java @@ -0,0 +1,49 @@ +/* + * 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.wali; + +/** + *

+ * Enumerates the valid types of things that can cause a + * {@link WriteAheadRepository} to update its state

+ */ +public enum UpdateType { + + /** + * Used when a new Record has been created + */ + CREATE, + /** + * Used when a Record has been updated in some way + */ + UPDATE, + /** + * Used to indicate that a Record has been deleted and should be removed + * from the Repository + */ + DELETE, + /** + * Used to indicate that a Record still exists but has been moved elsewhere, + * so that it is no longer maintained by the WALI instance + */ + SWAP_OUT, + /** + * Used to indicate that a Record that was previously Swapped Out is now + * being Swapped In + */ + SWAP_IN; +} diff --git a/commons/wali/src/main/java/org/wali/WriteAheadRepository.java b/commons/wali/src/main/java/org/wali/WriteAheadRepository.java new file mode 100644 index 0000000000..4567872f6d --- /dev/null +++ b/commons/wali/src/main/java/org/wali/WriteAheadRepository.java @@ -0,0 +1,122 @@ +/* + * 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.wali; + +import java.io.IOException; +import java.util.Collection; +import java.util.Set; + +/** + *

+ * A WriteAheadRepository is used to persist state that is otherwise kept + * in-memory. The Repository does not provide any query capability except to + * allow the data to be recovered upon restart of the system. + *

+ * + *

+ * A WriteAheadRepository operates by writing every update to an Edit Log. On + * restart, the data can be recovered by replaying all of the updates that are + * found in the Edit Log. This can, however, eventually result in very large + * Edit Logs, which can both take up massive amounts of disk space and take a + * long time to recover. In order to prevent this, the Repository provides a + * Checkpointing capability. This allows the current in-memory state of the + * Repository to be flushed to disk and the Edit Log to be deleted, thereby + * compacting the amount of space required to store the Repository. After a + * Checkpoint is performed, modifications are again written to an Edit Log. At + * this point, when the system is to be restored, it is restored by first + * loading the Checkpointed version of the Repository and then replaying the + * Edit Log. + *

+ * + *

+ * All implementations of WriteAheadRepository use one or more + * partitions to manage their Edit Logs. An implementation may require exactly + * one partition or may allow many partitions. + *

+ * + * @param + */ +public interface WriteAheadRepository { + + /** + *

+ * Updates the repository with the specified Records. The Collection must + * not contain multiple records with the same ID + *

+ * + * @param records the records to update + * @param forceSync specifies whether or not the Repository forces the data + * to be flushed to disk. If false, the data may be stored in Operating + * System buffers, which improves performance but could cause loss of data + * if power is lost or the Operating System crashes + * @throws IOException + * @throws IllegalArgumentException if multiple records within the given + * Collection have the same ID, as specified by {@link Record#getId()} + * method + * + * @return the index of the Partition that performed the update + */ + int update(Collection records, boolean forceSync) throws IOException; + + /** + *

+ * Recovers all records from the persisted state. This method must be called + * before any updates are issued to the Repository. + *

+ * + * @return + * @throws IOException + * @throws IllegalStateException if any updates have been issued against + * this Repository before this method is invoked + */ + Collection recoverRecords() throws IOException; + + /** + *

+ * Recovers all External Swap locations that were persisted. If this method + * is to be called, it must be called AFTER {@link #recoverRecords()} and + * BEFORE {@link update}. + *

+ * + * @return + * @throws IOException + */ + Set getRecoveredSwapLocations() throws IOException; + + /** + *

+ * Compacts the contents of the Repository so that rather than having a + * Snapshot and an Edit Log indicating many Updates to the Snapshot, the + * Snapshot is updated to contain the current state of the Repository, and + * the edit log is purged. + *

+ * + * + * @return the number of records that were written to the new snapshot + * @throws java.io.IOException + */ + int checkpoint() throws IOException; + + /** + *

+ * Causes the repository to checkpoint and then close any open resources. + *

+ * + * @throws IOException + */ + void shutdown() throws IOException; +} diff --git a/commons/wali/src/test/java/org/wali/DummyRecord.java b/commons/wali/src/test/java/org/wali/DummyRecord.java new file mode 100644 index 0000000000..e0f7f969aa --- /dev/null +++ b/commons/wali/src/test/java/org/wali/DummyRecord.java @@ -0,0 +1,61 @@ +/* + * 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.wali; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class DummyRecord { + + private final String id; + private final Map props; + private final UpdateType updateType; + + public DummyRecord(final String id, final UpdateType updateType) { + this.id = id; + this.props = new HashMap<>(); + this.updateType = updateType; + } + + public String getId() { + return id; + } + + public UpdateType getUpdateType() { + return updateType; + } + + public DummyRecord setProperties(final Map props) { + this.props.clear(); + this.props.putAll(props); + return this; + } + + public DummyRecord setProperty(final String name, final String value) { + this.props.put(name, value); + return this; + } + + public Map getProperties() { + return Collections.unmodifiableMap(this.props); + } + + public String getProperty(final String name) { + return props.get(name); + } +} diff --git a/commons/wali/src/test/java/org/wali/DummyRecordSerde.java b/commons/wali/src/test/java/org/wali/DummyRecordSerde.java new file mode 100644 index 0000000000..8cc7860610 --- /dev/null +++ b/commons/wali/src/test/java/org/wali/DummyRecordSerde.java @@ -0,0 +1,107 @@ +/* + * 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.wali; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.util.Map; + +public class DummyRecordSerde implements SerDe { + + public static final int NUM_UPDATE_TYPES = UpdateType.values().length; + private int throwIOEAfterNserializeEdits = -1; + private int serializeEditCount = 0; + + @Override + public void serializeEdit(final DummyRecord previousState, final DummyRecord record, final DataOutputStream out) throws IOException { + if (throwIOEAfterNserializeEdits >= 0 && (serializeEditCount++ >= throwIOEAfterNserializeEdits)) { + throw new IOException("Serialized " + (serializeEditCount - 1) + " records successfully, so now it's time to throw IOE"); + } + + out.write(record.getUpdateType().ordinal()); + out.writeUTF(record.getId()); + + if (record.getUpdateType() != UpdateType.DELETE) { + final Map props = record.getProperties(); + out.writeInt(props.size()); + for (final Map.Entry entry : props.entrySet()) { + out.writeUTF(entry.getKey()); + out.writeUTF(entry.getValue()); + } + } + } + + @Override + public void serializeRecord(final DummyRecord record, final DataOutputStream out) throws IOException { + serializeEdit(null, record, out); + } + + @Override + public DummyRecord deserializeRecord(final DataInputStream in, final int version) throws IOException { + final int index = in.read(); + if (index < 0) { + throw new EOFException(); + } + if (index >= NUM_UPDATE_TYPES) { + throw new IOException("Corrupt stream; got UpdateType value of " + index + " but there are only " + NUM_UPDATE_TYPES + " valid values"); + } + final UpdateType updateType = UpdateType.values()[index]; + final String id = in.readUTF(); + final DummyRecord record = new DummyRecord(id, updateType); + + if (record.getUpdateType() != UpdateType.DELETE) { + final int numProps = in.readInt(); + for (int i = 0; i < numProps; i++) { + final String key = in.readUTF(); + final String value = in.readUTF(); + record.setProperty(key, value); + } + } + return record; + } + + @Override + public Object getRecordIdentifier(final DummyRecord record) { + return record.getId(); + } + + @Override + public UpdateType getUpdateType(final DummyRecord record) { + return record.getUpdateType(); + } + + @Override + public DummyRecord deserializeEdit(final DataInputStream in, final Map currentVersion, final int version) throws IOException { + return deserializeRecord(in, version); + } + + @Override + public int getVersion() { + return 1; + } + + public void setThrowIOEAfterNSerializeEdits(final int n) { + this.throwIOEAfterNserializeEdits = n; + } + + @Override + public String getLocation(final DummyRecord record) { + return null; + } +} diff --git a/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java b/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java new file mode 100644 index 0000000000..57f3495b70 --- /dev/null +++ b/commons/wali/src/test/java/org/wali/TestMinimalLockingWriteAheadLog.java @@ -0,0 +1,298 @@ +/* + * 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.wali; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; + +import org.junit.Assert; +import org.junit.Test; + +public class TestMinimalLockingWriteAheadLog { + + @Test + public void testWrite() throws IOException, InterruptedException { + final int numPartitions = 8; + + final Path path = Paths.get("target/minimal-locking-repo"); + deleteRecursively(path.toFile()); + assertTrue(path.toFile().mkdirs()); + + final DummyRecordSerde serde = new DummyRecordSerde(); + final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection initialRecs = repo.recoverRecords(); + assertTrue(initialRecs.isEmpty()); + + final List threads = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + threads.add(new InsertThread(10000, 1000000 * i, repo)); + } + + final long start = System.nanoTime(); + for (final InsertThread thread : threads) { + thread.start(); + } + for (final InsertThread thread : threads) { + thread.join(); + } + final long nanos = System.nanoTime() - start; + final long millis = TimeUnit.MILLISECONDS.convert(nanos, TimeUnit.NANOSECONDS); + System.out.println("Took " + millis + " millis to insert 1,000,000 records each in its own transaction"); + repo.shutdown(); + + final WriteAheadRepository recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection recoveredRecords = recoverRepo.recoverRecords(); + assertFalse(recoveredRecords.isEmpty()); + assertEquals(100000, recoveredRecords.size()); + for (final DummyRecord record : recoveredRecords) { + final Map recoveredProps = record.getProperties(); + assertEquals(1, recoveredProps.size()); + assertEquals("B", recoveredProps.get("A")); + } + } + + @Test + public void testRecoverAfterIOException() throws IOException { + final int numPartitions = 5; + final Path path = Paths.get("target/minimal-locking-repo-test-recover-after-ioe"); + deleteRecursively(path.toFile()); + Files.createDirectories(path); + + final DummyRecordSerde serde = new DummyRecordSerde(); + final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection initialRecs = repo.recoverRecords(); + assertTrue(initialRecs.isEmpty()); + + serde.setThrowIOEAfterNSerializeEdits(7); // serialize the 2 transactions, then the first edit of the third transaction; then throw IOException + + final List firstTransaction = new ArrayList<>(); + firstTransaction.add(new DummyRecord("1", UpdateType.CREATE)); + firstTransaction.add(new DummyRecord("2", UpdateType.CREATE)); + firstTransaction.add(new DummyRecord("3", UpdateType.CREATE)); + + final List secondTransaction = new ArrayList<>(); + secondTransaction.add(new DummyRecord("1", UpdateType.UPDATE).setProperty("abc", "123")); + secondTransaction.add(new DummyRecord("2", UpdateType.UPDATE).setProperty("cba", "123")); + secondTransaction.add(new DummyRecord("3", UpdateType.UPDATE).setProperty("aaa", "123")); + + final List thirdTransaction = new ArrayList<>(); + thirdTransaction.add(new DummyRecord("1", UpdateType.DELETE)); + thirdTransaction.add(new DummyRecord("2", UpdateType.DELETE)); + + repo.update(firstTransaction, true); + repo.update(secondTransaction, true); + try { + repo.update(thirdTransaction, true); + Assert.fail("Did not throw IOException on third transaction"); + } catch (final IOException e) { + // expected behavior. + } + + repo.shutdown(); + + serde.setThrowIOEAfterNSerializeEdits(-1); + final WriteAheadRepository recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection recoveredRecords = recoverRepo.recoverRecords(); + assertFalse(recoveredRecords.isEmpty()); + assertEquals(3, recoveredRecords.size()); + + boolean record1 = false, record2 = false, record3 = false; + for (final DummyRecord record : recoveredRecords) { + switch (record.getId()) { + case "1": + record1 = true; + assertEquals("123", record.getProperty("abc")); + break; + case "2": + record2 = true; + assertEquals("123", record.getProperty("cba")); + break; + case "3": + record3 = true; + assertEquals("123", record.getProperty("aaa")); + break; + } + } + + assertTrue(record1); + assertTrue(record2); + assertTrue(record3); + } + + @Test + public void testCannotModifyLogAfterAllAreBlackListed() throws IOException { + final int numPartitions = 5; + final Path path = Paths.get("target/minimal-locking-repo-test-cannot-modify-after-all-blacklisted"); + deleteRecursively(path.toFile()); + Files.createDirectories(path); + + final DummyRecordSerde serde = new DummyRecordSerde(); + final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection initialRecs = repo.recoverRecords(); + assertTrue(initialRecs.isEmpty()); + + serde.setThrowIOEAfterNSerializeEdits(3); // serialize the first transaction, then fail on all subsequent transactions + + final List firstTransaction = new ArrayList<>(); + firstTransaction.add(new DummyRecord("1", UpdateType.CREATE)); + firstTransaction.add(new DummyRecord("2", UpdateType.CREATE)); + firstTransaction.add(new DummyRecord("3", UpdateType.CREATE)); + + final List secondTransaction = new ArrayList<>(); + secondTransaction.add(new DummyRecord("1", UpdateType.UPDATE).setProperty("abc", "123")); + secondTransaction.add(new DummyRecord("2", UpdateType.UPDATE).setProperty("cba", "123")); + secondTransaction.add(new DummyRecord("3", UpdateType.UPDATE).setProperty("aaa", "123")); + + final List thirdTransaction = new ArrayList<>(); + thirdTransaction.add(new DummyRecord("1", UpdateType.DELETE)); + thirdTransaction.add(new DummyRecord("2", UpdateType.DELETE)); + + repo.update(firstTransaction, true); + + try { + repo.update(secondTransaction, true); + Assert.fail("Did not throw IOException on second transaction"); + } catch (final IOException e) { + // expected behavior. + } + + for (int i = 0; i < 4; i++) { + try { + repo.update(thirdTransaction, true); + Assert.fail("Did not throw IOException on third transaction"); + } catch (final IOException e) { + // expected behavior. + } + } + + serde.setThrowIOEAfterNSerializeEdits(-1); + final List fourthTransaction = new ArrayList<>(); + fourthTransaction.add(new DummyRecord("1", UpdateType.DELETE)); + + try { + repo.update(fourthTransaction, true); + Assert.fail("Successfully updated repo for 4th transaction"); + } catch (final IOException e) { + // expected behavior + assertTrue(e.getMessage().contains("All Partitions have been blacklisted")); + } + + repo.shutdown(); + serde.setThrowIOEAfterNSerializeEdits(-1); + + final WriteAheadRepository recoverRepo = new MinimalLockingWriteAheadLog<>(path, numPartitions, serde, null); + final Collection recoveredRecords = recoverRepo.recoverRecords(); + assertFalse(recoveredRecords.isEmpty()); + assertEquals(3, recoveredRecords.size()); + } + + @Test + public void testStriping() throws IOException { + final int numPartitions = 6; + final Path path = Paths.get("target/minimal-locking-repo-striped"); + deleteRecursively(path.toFile()); + Files.createDirectories(path); + + final SortedSet paths = new TreeSet<>(); + paths.add(path.resolve("stripe-1")); + paths.add(path.resolve("stripe-2")); + + final DummyRecordSerde serde = new DummyRecordSerde(); + final WriteAheadRepository repo = new MinimalLockingWriteAheadLog<>(paths, numPartitions, serde, null); + final Collection initialRecs = repo.recoverRecords(); + assertTrue(initialRecs.isEmpty()); + + final InsertThread inserter = new InsertThread(100000, 0, repo); + inserter.run(); + + for (final Path partitionPath : paths) { + final File[] files = partitionPath.toFile().listFiles(new FileFilter() { + @Override + public boolean accept(File pathname) { + return pathname.getName().startsWith("partition"); + } + }); + assertEquals(3, files.length); + + for (final File file : files) { + final File[] journalFiles = file.listFiles(); + assertEquals(1, journalFiles.length); + } + } + + repo.checkpoint(); + + } + + private static class InsertThread extends Thread { + + private final List> records; + private final WriteAheadRepository repo; + + public InsertThread(final int numInsertions, final int startIndex, final WriteAheadRepository repo) { + records = new ArrayList<>(); + for (int i = 0; i < numInsertions; i++) { + final DummyRecord record = new DummyRecord(String.valueOf(i + startIndex), UpdateType.CREATE); + record.setProperty("A", "B"); + final List list = new ArrayList<>(); + list.add(record); + records.add(list); + } + this.repo = repo; + } + + @Override + public void run() { + try { + int counter = 0; + for (final List list : records) { + final boolean forceSync = (++counter == records.size()); + repo.update(list, forceSync); + } + } catch (IOException e) { + Assert.fail("Failed to update: " + e.toString()); + e.printStackTrace(); + } + } + } + + private void deleteRecursively(final File file) { + final File[] children = file.listFiles(); + if (children != null) { + for (final File child : children) { + deleteRecursively(child); + } + } + + file.delete(); + } +} diff --git a/extensions/file-authorization-provider/pom.xml b/extensions/file-authorization-provider/pom.xml new file mode 100644 index 0000000000..f8d823f6ea --- /dev/null +++ b/extensions/file-authorization-provider/pom.xml @@ -0,0 +1,90 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-parent + 0.0.1-SNAPSHOT + + + file-authorization-provider + 0.0.1-SNAPSHOT + Authorization Provider: File + + + + + src/main/resources + + + src/main/xsd + + + + + com.sun.tools.xjc.maven2 + maven-jaxb-plugin + + + current + + generate + + + src/main/xsd + + **/*.xsd + + org.apache.nifi.user.generated + + + + + ${project.build.directory}/generated-sources/jaxb + + + + + + + org.apache.nifi + nifi-api + [0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT) + + + org.apache.nifi + nifi-file-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-properties + [0.0.1-SNAPSHOT, 1.0.0-SNAPSHOT) + + + commons-lang + commons-lang + 2.6 + + + commons-codec + commons-codec + 1.10 + test + + + diff --git a/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java b/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java new file mode 100644 index 0000000000..55f59ba6c9 --- /dev/null +++ b/extensions/file-authorization-provider/src/main/java/org/apache/nifi/authorization/FileAuthorizationProvider.java @@ -0,0 +1,568 @@ +/* + * 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.authorization; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Set; +import javax.xml.XMLConstants; +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBElement; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; +import javax.xml.bind.Unmarshaller; +import javax.xml.transform.stream.StreamSource; +import javax.xml.validation.Schema; +import javax.xml.validation.SchemaFactory; +import org.apache.nifi.authorization.annotation.AuthorityProviderContext; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException; +import org.apache.nifi.authorization.exception.ProviderCreationException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.file.FileUtils; +import org.apache.nifi.user.generated.ObjectFactory; +import org.apache.nifi.user.generated.Role; +import org.apache.nifi.user.generated.User; +import org.apache.nifi.user.generated.Users; +import org.apache.nifi.util.NiFiProperties; +import org.apache.commons.lang.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.xml.sax.SAXException; + +/** + * Provides identity checks and grants authorities. + */ +public class FileAuthorizationProvider implements AuthorityProvider { + + private static final Logger logger = LoggerFactory.getLogger(FileAuthorizationProvider.class); + private static final String USERS_XSD = "/users.xsd"; + private static final String JAXB_GENERATED_PATH = "org.apache.nifi.user.generated"; + private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext(); + + /** + * Load the JAXBContext. + */ + private static JAXBContext initializeJaxbContext() { + try { + return JAXBContext.newInstance(JAXB_GENERATED_PATH, FileAuthorizationProvider.class.getClassLoader()); + } catch (JAXBException e) { + throw new RuntimeException("Unable to create JAXBContext."); + } + } + + private NiFiProperties properties; + private File usersFile; + private File restoreUsersFile; + private Users users; + private final Set defaultAuthorities = new HashSet<>(); + + @Override + public void initialize(final AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException { + } + + @Override + public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException { + try { + final String usersFilePath = configurationContext.getProperty("Authorized Users File"); + if (usersFilePath == null || usersFilePath.trim().isEmpty()) { + throw new ProviderCreationException("The authorized users file must be specified."); + } + + // the users file instance will never be null because a default is used + usersFile = new File(usersFilePath); + final File usersFileDirectory = usersFile.getParentFile(); + + // the restore directory is optional and may be null + final File restoreDirectory = properties.getRestoreDirectory(); + + if (restoreDirectory != null) { + + // sanity check that restore directory is a directory, creating it if necessary + FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory); + + // check that restore directory is not the same as the primary directory + if (usersFileDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) { + throw new ProviderCreationException(String.format("Authorized User's directory '%s' is the same as restore directory '%s' ", + usersFileDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath())); + } + + // the restore copy will have same file name, but reside in a different directory + restoreUsersFile = new File(restoreDirectory, usersFile.getName()); + + // sync the primary copy with the restore copy + try { + FileUtils.syncWithRestore(usersFile, restoreUsersFile, logger); + } catch (final IOException | IllegalStateException ioe) { + throw new ProviderCreationException(ioe); + } + + } + + // load the users from the specified file + if (usersFile.exists()) { + // find the schema + final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); + final Schema schema = schemaFactory.newSchema(FileAuthorizationProvider.class.getResource(USERS_XSD)); + + // attempt to unmarshal + final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller(); + unmarshaller.setSchema(schema); + final JAXBElement element = unmarshaller.unmarshal(new StreamSource(usersFile), Users.class); + users = element.getValue(); + } else { + final ObjectFactory objFactory = new ObjectFactory(); + users = objFactory.createUsers(); + } + + // attempt to load a default roles + final String rawDefaultAuthorities = configurationContext.getProperty("Default User Roles"); + if (StringUtils.isNotBlank(rawDefaultAuthorities)) { + final Set invalidDefaultAuthorities = new HashSet<>(); + + // validate the specified authorities + final String[] rawDefaultAuthorityList = rawDefaultAuthorities.split(","); + for (String rawAuthority : rawDefaultAuthorityList) { + rawAuthority = rawAuthority.trim(); + final Authority authority = Authority.valueOfAuthority(rawAuthority); + if (authority == null) { + invalidDefaultAuthorities.add(rawAuthority); + } else { + defaultAuthorities.add(rawAuthority); + } + } + + // report any unrecognized authorities + if (!invalidDefaultAuthorities.isEmpty()) { + logger.warn(String.format("The following default role(s) '%s' were not recognized. Possible values: %s.", + StringUtils.join(invalidDefaultAuthorities, ", "), StringUtils.join(Authority.getRawAuthorities(), ", "))); + } + } + } catch (IOException | ProviderCreationException | SAXException | JAXBException e) { + throw new ProviderCreationException(e); + } + + } + + @Override + public void preDestruction() { + } + + /** + * Determines if this provider has a default role. + * + * @return + */ + private boolean hasDefaultRoles() { + return !defaultAuthorities.isEmpty(); + } + + /** + * Determines if the specified dn is known to this authority provider. When + * this provider is configured to have default role(s), all dn are + * considered to exist. + * + * @param dn + * @return True if he dn is known, false otherwise + */ + @Override + public boolean doesDnExist(String dn) throws AuthorityAccessException { + if (hasDefaultRoles()) { + return true; + } + + final User user = getUser(dn); + return user != null; + } + + /** + * Loads the authorities for the specified user. If this provider is + * configured for default user role(s) and a non existent dn is specified, a + * new user will be automatically created with the default role(s). + * + * @param dn + * @return + * @throws UnknownIdentityException + * @throws AuthorityAccessException + */ + @Override + public synchronized Set getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException { + final Set authorities = EnumSet.noneOf(Authority.class); + + // get the user + final User user = getUser(dn); + + // ensure the user was located + if (user == null) { + if (hasDefaultRoles()) { + logger.debug(String.format("User DN not found: %s. Creating new user with default roles.", dn)); + + // create the user (which will automatically add any default authorities) + addUser(dn, null); + + // get the authorities for the newly created user + authorities.addAll(getAuthorities(dn)); + } else { + throw new UnknownIdentityException(String.format("User DN not found: %s.", dn)); + } + } else { + // create the authorities that this user has + for (final Role role : user.getRole()) { + authorities.add(Authority.valueOfAuthority(role.getName())); + } + } + + return authorities; + } + + /** + * Adds the specified authorities to the specified user. Regardless of + * whether this provider is configured for a default user role, when a non + * existent dn is specified, an UnknownIdentityException will be thrown. + * + * @param dn + * @param authorities + * @throws UnknownIdentityException + * @throws AuthorityAccessException + */ + @Override + public synchronized void setAuthorities(String dn, Set authorities) throws UnknownIdentityException, AuthorityAccessException { + // get the user + final User user = getUser(dn); + + // ensure the user was located + if (user == null) { + throw new UnknownIdentityException(String.format("User DN not found: %s.", dn)); + } + + // add the user authorities + setUserAuthorities(user, authorities); + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + /** + * Adds the specified authorities to the specified user. + * + * @param user + * @param authorities + */ + private void setUserAuthorities(final User user, final Set authorities) { + // clear the existing rules + user.getRole().clear(); + + // set the new roles + final ObjectFactory objFactory = new ObjectFactory(); + for (final Authority authority : authorities) { + final Role role = objFactory.createRole(); + role.setName(authority.toString()); + + // add the new role + user.getRole().add(role); + } + } + + /** + * Adds the specified user. If this provider is configured with default + * role(s) they will be added to the new user. + * + * @param dn + * @param group + * @throws UnknownIdentityException + * @throws AuthorityAccessException + */ + @Override + public synchronized void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException { + final User user = getUser(dn); + + // ensure the user doesn't already exist + if (user != null) { + throw new IdentityAlreadyExistsException(String.format("User DN already exists: %s", dn)); + } + + // create the new user + final ObjectFactory objFactory = new ObjectFactory(); + final User newUser = objFactory.createUser(); + + // set the user properties + newUser.setDn(dn); + newUser.setGroup(group); + + // add default roles if appropriate + if (hasDefaultRoles()) { + for (final String authority : defaultAuthorities) { + Role role = objFactory.createRole(); + role.setName(authority); + + // add the role + newUser.getRole().add(role); + } + } + + // add the user + users.getUser().add(newUser); + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + /** + * Gets the users for the specified authority. + * + * @param authority + * @return + * @throws AuthorityAccessException + */ + @Override + public synchronized Set getUsers(Authority authority) throws AuthorityAccessException { + final Set userSet = new HashSet<>(); + for (final User user : users.getUser()) { + for (final Role role : user.getRole()) { + if (role.getName().equals(authority.toString())) { + userSet.add(user.getDn()); + } + } + } + return userSet; + } + + /** + * Removes the specified user. Regardless of whether this provider is + * configured for a default user role, when a non existent dn is specified, + * an UnknownIdentityException will be thrown. + * + * @param dn + * @throws UnknownIdentityException + * @throws AuthorityAccessException + */ + @Override + public synchronized void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + // get the user + final User user = getUser(dn); + + // ensure the user was located + if (user == null) { + throw new UnknownIdentityException(String.format("User DN not found: %s.", dn)); + } + + // remove the specified user + users.getUser().remove(user); + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + @Override + public void setUsersGroup(Set dns, String group) throws UnknownIdentityException, AuthorityAccessException { + final Collection groupedUsers = new HashSet<>(); + + // get the specified users + for (final String dn : dns) { + // get the user + final User user = getUser(dn); + + // ensure the user was located + if (user == null) { + throw new UnknownIdentityException(String.format("User DN not found: %s.", dn)); + } + + groupedUsers.add(user); + } + + // update each user group + for (final User user : groupedUsers) { + user.setGroup(group); + } + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + @Override + public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + // get the user + final User user = getUser(dn); + + // ensure the user was located + if (user == null) { + throw new UnknownIdentityException(String.format("User DN not found: %s.", dn)); + } + + // remove the users group + user.setGroup(null); + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + @Override + public void ungroup(String group) throws AuthorityAccessException { + // get the user group + final Collection userGroup = getUserGroup(group); + + // ensure the user group was located + if (userGroup == null) { + return; + } + + // update each user group + for (final User user : userGroup) { + user.setGroup(null); + } + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + @Override + public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + // get the user + final User user = getUser(dn); + + // ensure the user was located + if (user == null) { + throw new UnknownIdentityException(String.format("User DN not found: %s.", dn)); + } + + return user.getGroup(); + } + + @Override + public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException { + // get the user group + final Collection userGroup = getUserGroup(group); + + // ensure the user group was located + if (userGroup == null) { + throw new UnknownIdentityException(String.format("User group not found: %s.", group)); + } + + // remove each user in the group + for (final User user : userGroup) { + users.getUser().remove(user); + } + + try { + // save the file + save(); + } catch (Exception e) { + throw new AuthorityAccessException(e.getMessage(), e); + } + } + + /** + * Locates the user with the specified DN. + * + * @param dn + * @return + */ + private User getUser(String dn) throws UnknownIdentityException { + // ensure the DN was specified + if (dn == null) { + throw new UnknownIdentityException("User DN not specified."); + } + + // attempt to get the user and ensure it was located + User desiredUser = null; + for (final User user : users.getUser()) { + if (dn.equalsIgnoreCase(user.getDn())) { + desiredUser = user; + break; + } + } + + return desiredUser; + } + + /** + * Locates all users that are part of the specified group. + * + * @param group + * @return + * @throws UnknownIdentityException + */ + private Collection getUserGroup(String group) throws UnknownIdentityException { + // ensure the DN was specified + if (group == null) { + throw new UnknownIdentityException("User group not specified."); + } + + // get all users with this group + Collection userGroup = null; + for (final User user : users.getUser()) { + if (group.equals(user.getGroup())) { + if (userGroup == null) { + userGroup = new HashSet<>(); + } + userGroup.add(user); + } + } + + return userGroup; + } + + /** + * Saves the users file. + * + * @throws Exception + */ + private void save() throws Exception { + final Marshaller marshaller = JAXB_CONTEXT.createMarshaller(); + marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, Boolean.TRUE); + + // save users to restore directory before primary directory + if (restoreUsersFile != null) { + marshaller.marshal(users, restoreUsersFile); + } + + // save users to primary directory + marshaller.marshal(users, usersFile); + } + + @AuthorityProviderContext + public void setNiFiProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider b/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider new file mode 100755 index 0000000000..93d2941bc4 --- /dev/null +++ b/extensions/file-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.authorization.FileAuthorizationProvider diff --git a/extensions/file-authorization-provider/src/main/xsd/users.xsd b/extensions/file-authorization-provider/src/main/xsd/users.xsd new file mode 100644 index 0000000000..4ee1e17226 --- /dev/null +++ b/extensions/file-authorization-provider/src/main/xsd/users.xsd @@ -0,0 +1,64 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java b/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java new file mode 100644 index 0000000000..3d0196dbe1 --- /dev/null +++ b/extensions/file-authorization-provider/src/test/java/org/apache/nifi/authorization/FileAuthorizationProviderTest.java @@ -0,0 +1,127 @@ +/* + * 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.authorization; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import org.apache.nifi.authorization.exception.ProviderCreationException; +import org.apache.nifi.file.FileUtils; +import org.apache.nifi.util.NiFiProperties; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Mockito.*; +import static org.junit.Assert.*; +import org.junit.Ignore; +import org.mockito.Mockito; + +@Ignore +public class FileAuthorizationProviderTest { + + private FileAuthorizationProvider provider; + + private File primary; + + private File restore; + + private NiFiProperties mockProperties; + + private AuthorityProviderConfigurationContext mockConfigurationContext; + + @Before + public void setup() throws IOException { + + primary = new File("target/primary/users.txt"); + restore = new File("target/restore/users.txt"); + + System.out.println("absolute path: " + primary.getAbsolutePath()); + + mockProperties = mock(NiFiProperties.class); + when(mockProperties.getRestoreDirectory()).thenReturn(restore.getParentFile()); + + mockConfigurationContext = mock(AuthorityProviderConfigurationContext.class); + when(mockConfigurationContext.getProperty(Mockito.eq("Authorized Users File"))).thenReturn(primary.getPath()); + + provider = new FileAuthorizationProvider(); + provider.setNiFiProperties(mockProperties); + provider.initialize(null); + } + + @After + public void cleanup() throws Exception { + deleteFile(primary); + deleteFile(restore); + } + + private boolean deleteFile(final File file) { + if(file.isDirectory()) { + FileUtils.deleteFilesInDir(file, null, null, true, true); + } + return FileUtils.deleteFile(file, null, 10); + } + + @Test + public void testPostContructionWhenRestoreDoesNotExist() throws Exception { + + byte[] primaryBytes = "".getBytes(); + FileOutputStream fos = new FileOutputStream(primary); + fos.write(primaryBytes); + fos.close(); + + provider.onConfigured(mockConfigurationContext); + assertEquals(primary.length(), restore.length()); + } + + @Test + public void testPostContructionWhenPrimaryDoesNotExist() throws Exception { + + byte[] restoreBytes = "".getBytes(); + FileOutputStream fos = new FileOutputStream(restore); + fos.write(restoreBytes); + fos.close(); + + provider.onConfigured(mockConfigurationContext); + assertEquals(restore.length(), primary.length()); + + } + + @Test(expected = ProviderCreationException.class) + public void testPostContructionWhenPrimaryDifferentThanRestore() throws Exception { + + byte[] primaryBytes = "".getBytes(); + FileOutputStream fos = new FileOutputStream(primary); + fos.write(primaryBytes); + fos.close(); + + byte[] restoreBytes = "".getBytes(); + fos = new FileOutputStream(restore); + fos.write(restoreBytes); + fos.close(); + + provider.onConfigured(mockConfigurationContext); + } + + @Test + public void testPostContructionWhenPrimaryAndBackupDoNotExist() throws Exception { + + provider.onConfigured(mockConfigurationContext); + assertEquals(0, restore.length()); + assertEquals(restore.length(), primary.length()); + } + +} diff --git a/misc/build-order.sh b/misc/build-order.sh new file mode 100755 index 0000000000..855321a312 --- /dev/null +++ b/misc/build-order.sh @@ -0,0 +1,79 @@ +#MAVEN_FLAGS="-Dmaven.test.skip=true" +MAVEN_FLAGS="" + +cd misc/nar-maven-plugin && \ +mvn $MAVEN_FLAGS install && \ +cd ../../commons/nifi-parent && \ +mvn $MAVEN_FLAGS install && \ +cd ../../nifi-api && \ +mvn $MAVEN_FLAGS install && \ +cd ../commons/ && \ +cd nifi-stream-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../wali && \ +mvn $MAVEN_FLAGS install && \ +cd ../flowfile-packager && \ +mvn $MAVEN_FLAGS install && \ +cd ../core-flowfile-attributes && \ +mvn $MAVEN_FLAGS install && \ +cd ../data-provenance-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../naive-search-ring-buffer && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-expression-language && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-file-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-logging-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-properties && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-security-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-socket-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../nifi-web-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../processor-utilities && \ +mvn $MAVEN_FLAGS install && \ +cd ../remote-communications-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../search-utils && \ +mvn $MAVEN_FLAGS install && \ +cd ../../extensions/file-authorization-provider && \ +mvn $MAVEN_FLAGS install && \ +cd ../../nifi-mock && \ +mvn $MAVEN_FLAGS install && \ +cd ../nar-bundles/ && \ +cd nar-container-common && \ +mvn $MAVEN_FLAGS install && \ +cd ../jetty-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../standard-services-api-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../ssl-context-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../distributed-cache-services-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../standard-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../hadoop-libraries-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../hadoop-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../volatile-provenance-repository-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../persistent-provenance-repository-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../framework-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../execute-script-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../monitor-threshold-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../update-attribute-bundle && \ +mvn $MAVEN_FLAGS install && \ +cd ../../assemblies/nifi +mvn assembly:assembly diff --git a/misc/nar-maven-plugin/pom.xml b/misc/nar-maven-plugin/pom.xml new file mode 100644 index 0000000000..3888df39ff --- /dev/null +++ b/misc/nar-maven-plugin/pom.xml @@ -0,0 +1,83 @@ + + + + 4.0.0 + org.apache.nifi + nar-maven-plugin + 0.0.1-SNAPSHOT + maven-plugin + Apache NiFi NAR Plugin + + UTF-8 + + + install + + + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + + + + maven-jar-plugin + 2.5 + + + org.apache.maven.plugins + maven-plugin-plugin + 3.3 + + + + + + org.apache.maven + maven-plugin-api + 2.0.11 + + + org.apache.maven.plugins + maven-dependency-plugin + 2.9 + maven-plugin + + + + org.apache.maven.plugins + maven-jar-plugin + 2.5 + + + org.apache.maven.plugin-tools + maven-plugin-annotations + 3.3 + + + + + nifi-releases + ${nifi.repo.url} + + + nifi-snapshots + ${nifi.snapshot.repo.url} + + + diff --git a/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java b/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java new file mode 100644 index 0000000000..263fe8820e --- /dev/null +++ b/misc/nar-maven-plugin/src/main/java/nifi/NarMojo.java @@ -0,0 +1,610 @@ +/* + * 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 nifi; + +import java.io.File; +import java.io.IOException; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import org.apache.maven.archiver.MavenArchiveConfiguration; +import org.apache.maven.archiver.MavenArchiver; +import org.apache.maven.artifact.Artifact; +import org.apache.maven.artifact.DependencyResolutionRequiredException; +import org.apache.maven.artifact.factory.ArtifactFactory; +import org.apache.maven.artifact.installer.ArtifactInstaller; +import org.apache.maven.artifact.metadata.ArtifactMetadataSource; +import org.apache.maven.artifact.repository.ArtifactRepository; +import org.apache.maven.artifact.repository.ArtifactRepositoryFactory; +import org.apache.maven.artifact.resolver.ArtifactCollector; +import org.apache.maven.artifact.resolver.ArtifactNotFoundException; +import org.apache.maven.artifact.resolver.ArtifactResolutionException; +import org.apache.maven.artifact.resolver.ArtifactResolver; +import org.apache.maven.plugin.AbstractMojo; +import org.apache.maven.plugin.MojoExecutionException; +import org.apache.maven.plugin.MojoFailureException; +import org.apache.maven.plugin.dependency.utils.DependencyStatusSets; +import org.apache.maven.plugin.dependency.utils.DependencyUtil; +import org.apache.maven.plugin.dependency.utils.filters.DestFileFilter; +import org.apache.maven.plugin.dependency.utils.resolvers.ArtifactsResolver; +import org.apache.maven.plugin.dependency.utils.resolvers.DefaultArtifactsResolver; +import org.apache.maven.plugin.dependency.utils.translators.ArtifactTranslator; +import org.apache.maven.plugin.dependency.utils.translators.ClassifierTypeTranslator; +import org.apache.maven.plugins.annotations.LifecyclePhase; +import org.apache.maven.plugins.annotations.Mojo; +import org.apache.maven.plugins.annotations.Parameter; +import org.apache.maven.plugins.annotations.ResolutionScope; +import org.apache.maven.project.MavenProject; +import org.apache.maven.execution.MavenSession; +import org.apache.maven.plugins.annotations.Component; +import org.apache.maven.project.MavenProjectHelper; +import org.apache.maven.shared.artifact.filter.collection.ArtifactFilterException; +import org.apache.maven.shared.artifact.filter.collection.ArtifactIdFilter; +import org.apache.maven.shared.artifact.filter.collection.ArtifactsFilter; +import org.apache.maven.shared.artifact.filter.collection.ClassifierFilter; +import org.apache.maven.shared.artifact.filter.collection.FilterArtifacts; +import org.apache.maven.shared.artifact.filter.collection.GroupIdFilter; +import org.apache.maven.shared.artifact.filter.collection.ScopeFilter; +import org.apache.maven.shared.artifact.filter.collection.ProjectTransitivityFilter; +import org.apache.maven.shared.artifact.filter.collection.TypeFilter; +import org.codehaus.plexus.archiver.ArchiverException; +import org.codehaus.plexus.archiver.jar.JarArchiver; +import org.codehaus.plexus.archiver.jar.ManifestException; +import org.codehaus.plexus.archiver.manager.ArchiverManager; +import org.codehaus.plexus.util.FileUtils; +import org.codehaus.plexus.util.StringUtils; + +/** + * Packages the current project as an Apache NiFi Archive (NAR). + * + * The following code is derived from maven-dependencies-plugin and + * maven-jar-plugin. The functionality of CopyDependenciesMojo and JarMojo was + * simplified to the use case of NarMojo. + * + */ +@Mojo(name = "nar", defaultPhase = LifecyclePhase.PACKAGE, threadSafe = false, requiresDependencyResolution = ResolutionScope.RUNTIME) +public class NarMojo extends AbstractMojo { + + private static final String[] DEFAULT_EXCLUDES = new String[]{"**/package.html"}; + private static final String[] DEFAULT_INCLUDES = new String[]{"**/**"}; + + /** + * POM + * + */ + @Parameter(property = "project", readonly = true, required = true) + protected MavenProject project; + + @Parameter(property = "session", readonly = true, required = true) + protected MavenSession session; + + /** + * List of files to include. Specified as fileset patterns. + */ + @Parameter(property = "includes") + protected String[] includes; + /** + * List of files to exclude. Specified as fileset patterns. + */ + @Parameter(property = "excludes") + protected String[] excludes; + /** + * Name of the generated NAR. + * + */ + @Parameter(alias = "narName", property = "nar.finalName", defaultValue = "${project.build.finalName}", required = true) + protected String finalName; + + /** + * The Jar archiver. + * + * \@\component role="org.codehaus.plexus.archiver.Archiver" roleHint="jar" + */ + @Component(role = org.codehaus.plexus.archiver.Archiver.class, hint = "jar") + private JarArchiver jarArchiver; + /** + * The archive configuration to use. + * + * See the + * documentation for Maven Archiver. + * + */ + @Parameter(property = "archive") + protected final MavenArchiveConfiguration archive = new MavenArchiveConfiguration(); + /** + * Path to the default MANIFEST file to use. It will be used if + * useDefaultManifestFile is set to true. + * + */ + @Parameter(property = "defaultManifestFiles", defaultValue = "${project.build.outputDirectory}/META-INF/MANIFEST.MF", readonly = true, required = true) + protected File defaultManifestFile; + + /** + * Set this to true to enable the use of the + * defaultManifestFile. + * + * @since 2.2 + */ + @Parameter(property = "nar.useDefaultManifestFile", defaultValue = "false") + protected boolean useDefaultManifestFile; + + @Component + protected MavenProjectHelper projectHelper; + + /** + * Whether creating the archive should be forced. + * + */ + @Parameter(property = "nar.forceCreation", defaultValue = "false") + protected boolean forceCreation; + + /** + * Classifier to add to the artifact generated. If given, the artifact will + * be an attachment instead. + * + */ + @Parameter(property = "classifier") + protected String classifier; + + @Component + protected ArtifactInstaller installer; + + @Component + protected ArtifactRepositoryFactory repositoryFactory; + + /** + * This only applies if the classifier parameter is used. + * + */ + @Parameter(property = "mdep.failOnMissingClassifierArtifact", defaultValue = "true", required = false) + protected boolean failOnMissingClassifierArtifact = true; + + /** + * Comma Separated list of Types to include. Empty String indicates include + * everything (default). + * + */ + @Parameter(property = "includeTypes", required = false) + protected String includeTypes; + + /** + * Comma Separated list of Types to exclude. Empty String indicates don't + * exclude anything (default). + * + */ + @Parameter(property = "excludeTypes", required = false) + protected String excludeTypes; + + /** + * Scope to include. An Empty string indicates all scopes (default). + * + */ + @Parameter(property = "includeScope", required = false) + protected String includeScope; + + /** + * Scope to exclude. An Empty string indicates no scopes (default). + * + */ + @Parameter(property = "excludeScope", required = false) + protected String excludeScope; + + /** + * Comma Separated list of Classifiers to include. Empty String indicates + * include everything (default). + * + */ + @Parameter(property = "includeClassifiers", required = false) + protected String includeClassifiers; + + /** + * Comma Separated list of Classifiers to exclude. Empty String indicates + * don't exclude anything (default). + * + */ + @Parameter(property = "excludeClassifiers", required = false) + protected String excludeClassifiers; + + /** + * Specify classifier to look for. Example: sources + * + */ + @Parameter(property = "classifier", required = false) + protected String copyDepClassifier; + + /** + * Specify type to look for when constructing artifact based on classifier. + * Example: java-source,jar,war, nar + * + */ + @Parameter(property = "type", required = false, defaultValue = "nar") + protected String type; + + /** + * Comma separated list of Artifact names too exclude. + * + */ + @Parameter(property = "excludeArtifacts", required = false) + protected String excludeArtifactIds; + + /** + * Comma separated list of Artifact names to include. + * + */ + @Parameter(property = "includeArtifacts", required = false) + protected String includeArtifactIds; + + /** + * Comma separated list of GroupId Names to exclude. + * + */ + @Parameter(property = "excludeArtifacts", required = false) + protected String excludeGroupIds; + + /** + * Comma separated list of GroupIds to include. + * + */ + @Parameter(property = "includeGroupIds", required = false) + protected String includeGroupIds; + + /** + * Directory to store flag files + * + */ + @Parameter(property = "markersDirectory", required = false, defaultValue = "${project.build.directory}/dependency-maven-plugin-markers") + protected File markersDirectory; + + /** + * Overwrite release artifacts + * + */ + @Parameter(property = "overWriteReleases", required = false) + protected boolean overWriteReleases; + + /** + * Overwrite snapshot artifacts + * + */ + @Parameter(property = "overWriteSnapshots", required = false) + protected boolean overWriteSnapshots; + + /** + * Overwrite artifacts that don't exist or are older than the source. + * + */ + @Parameter(property = "overWriteIfNewer", required = false, defaultValue = "true") + protected boolean overWriteIfNewer; + + /** + * Used to look up Artifacts in the remote repository. + */ + @Component + protected ArtifactFactory factory; + + /** + * Used to look up Artifacts in the remote repository. + * + */ + @Component + protected ArtifactResolver resolver; + + /** + * Artifact collector, needed to resolve dependencies. + * + */ + @Component(role = org.apache.maven.artifact.resolver.ArtifactCollector.class) + protected ArtifactCollector artifactCollector; + + @Component(role = org.apache.maven.artifact.metadata.ArtifactMetadataSource.class) + protected ArtifactMetadataSource artifactMetadataSource; + + /** + * Location of the local repository. + * + */ + @Parameter(property = "localRepository", required = true, readonly = true) + protected ArtifactRepository local; + + /** + * List of Remote Repositories used by the resolver + * + */ + @Parameter(property = "project.remoteArtifactRepositories", required = true, readonly = true) + protected List remoteRepos; + + /** + * To look up Archiver/UnArchiver implementations + * + */ + @Component + protected ArchiverManager archiverManager; + + /** + * Contains the full list of projects in the reactor. + * + */ + @Parameter(property = "reactorProjects", required = true, readonly = true) + protected List reactorProjects; + + /** + * If the plugin should be silent. + * + */ + @Parameter(property = "silent", required = false, defaultValue = "false") + public boolean silent; + + /** + * Output absolute filename for resolved artifacts + * + */ + @Parameter(property = "outputAbsoluteArtifactFilename", defaultValue = "false", required = false) + protected boolean outputAbsoluteArtifactFilename; + + @Override + public void execute() throws MojoExecutionException, MojoFailureException { + copyDependencies(); + makeNar(); + } + + private void copyDependencies() throws MojoExecutionException { + DependencyStatusSets dss = getDependencySets(this.failOnMissingClassifierArtifact); + Set artifacts = dss.getResolvedDependencies(); + + for (Object artifactObj : artifacts) { + copyArtifact((Artifact) artifactObj); + } + + artifacts = dss.getSkippedDependencies(); + for (Object artifactOjb : artifacts) { + Artifact artifact = (Artifact) artifactOjb; + getLog().info(artifact.getFile().getName() + " already exists in destination."); + } + } + + protected void copyArtifact(Artifact artifact) throws MojoExecutionException { + String destFileName = DependencyUtil.getFormattedFileName(artifact, false); + final File destDir = DependencyUtil.getFormattedOutputDirectory(false, false, false, false, false, getDependenciesDirectory(), artifact); + final File destFile = new File(destDir, destFileName); + copyFile(artifact.getFile(), destFile); + } + + protected Artifact getResolvedPomArtifact(Artifact artifact) { + Artifact pomArtifact = this.factory.createArtifact(artifact.getGroupId(), artifact.getArtifactId(), artifact.getVersion(), "", "pom"); + // Resolve the pom artifact using repos + try { + this.resolver.resolve(pomArtifact, this.remoteRepos, this.local); + } catch (ArtifactResolutionException | ArtifactNotFoundException e) { + getLog().info(e.getMessage()); + } + return pomArtifact; + } + + protected ArtifactsFilter getMarkedArtifactFilter() { + return new DestFileFilter(this.overWriteReleases, this.overWriteSnapshots, this.overWriteIfNewer, false, false, false, false, false, getDependenciesDirectory()); + } + + protected DependencyStatusSets getDependencySets(boolean stopOnFailure) throws MojoExecutionException { + // add filters in well known order, least specific to most specific + FilterArtifacts filter = new FilterArtifacts(); + + filter.addFilter(new ProjectTransitivityFilter(project.getDependencyArtifacts(), false)); + filter.addFilter(new ScopeFilter(this.includeScope, this.excludeScope)); + filter.addFilter(new TypeFilter(this.includeTypes, this.excludeTypes)); + filter.addFilter(new ClassifierFilter(this.includeClassifiers, this.excludeClassifiers)); + filter.addFilter(new GroupIdFilter(this.includeGroupIds, this.excludeGroupIds)); + filter.addFilter(new ArtifactIdFilter(this.includeArtifactIds, this.excludeArtifactIds)); + + // explicitly filter our nar dependencies + filter.addFilter(new TypeFilter("", "nar")); + + // start with all artifacts. + Set artifacts = project.getArtifacts(); + + // perform filtering + try { + artifacts = filter.filter(artifacts); + } catch (ArtifactFilterException e) { + throw new MojoExecutionException(e.getMessage(), e); + } + + // transform artifacts if classifier is set + final DependencyStatusSets status; + if (StringUtils.isNotEmpty(copyDepClassifier)) { + status = getClassifierTranslatedDependencies(artifacts, stopOnFailure); + } else { + status = filterMarkedDependencies(artifacts); + } + + return status; + } + + protected DependencyStatusSets getClassifierTranslatedDependencies(Set artifacts, boolean stopOnFailure) throws MojoExecutionException { + Set unResolvedArtifacts = new HashSet(); + Set resolvedArtifacts = artifacts; + DependencyStatusSets status = new DependencyStatusSets(); + + // possibly translate artifacts into a new set of artifacts based on the + // classifier and type + // if this did something, we need to resolve the new artifacts + if (StringUtils.isNotEmpty(copyDepClassifier)) { + ArtifactTranslator translator = new ClassifierTypeTranslator(this.copyDepClassifier, this.type, this.factory); + artifacts = translator.translate(artifacts, getLog()); + + status = filterMarkedDependencies(artifacts); + + // the unskipped artifacts are in the resolved set. + artifacts = status.getResolvedDependencies(); + + // resolve the rest of the artifacts + ArtifactsResolver artifactsResolver = new DefaultArtifactsResolver(this.resolver, this.local, + this.remoteRepos, stopOnFailure); + resolvedArtifacts = artifactsResolver.resolve(artifacts, getLog()); + + // calculate the artifacts not resolved. + unResolvedArtifacts.addAll(artifacts); + unResolvedArtifacts.removeAll(resolvedArtifacts); + } + + // return a bean of all 3 sets. + status.setResolvedDependencies(resolvedArtifacts); + status.setUnResolvedDependencies(unResolvedArtifacts); + + return status; + } + + protected DependencyStatusSets filterMarkedDependencies(Set artifacts) throws MojoExecutionException { + // remove files that have markers already + FilterArtifacts filter = new FilterArtifacts(); + filter.clearFilters(); + filter.addFilter(getMarkedArtifactFilter()); + + Set unMarkedArtifacts; + try { + unMarkedArtifacts = filter.filter(artifacts); + } catch (ArtifactFilterException e) { + throw new MojoExecutionException(e.getMessage(), e); + } + + // calculate the skipped artifacts + Set skippedArtifacts = new HashSet(); + skippedArtifacts.addAll(artifacts); + skippedArtifacts.removeAll(unMarkedArtifacts); + + return new DependencyStatusSets(unMarkedArtifacts, null, skippedArtifacts); + } + + protected void copyFile(File artifact, File destFile) throws MojoExecutionException { + try { + getLog().info("Copying " + (this.outputAbsoluteArtifactFilename ? artifact.getAbsolutePath() : artifact.getName()) + " to " + destFile); + FileUtils.copyFile(artifact, destFile); + } catch (Exception e) { + throw new MojoExecutionException("Error copying artifact from " + artifact + " to " + destFile, e); + } + } + + private File getClassesDirectory() { + final File outputDirectory = new File(project.getBasedir(), "target"); + return new File(outputDirectory, "classes"); + } + + private File getDependenciesDirectory() { + return new File(getClassesDirectory(), "META-INF/dependencies"); + } + + private void makeNar() throws MojoExecutionException { + File narFile = createArchive(); + + if (classifier != null) { + projectHelper.attachArtifact(project, "nar", classifier, narFile); + } else { + project.getArtifact().setFile(narFile); + } + } + + public File createArchive() throws MojoExecutionException { + final File outputDirectory = new File(project.getBasedir(), "target"); + File narFile = getNarFile(outputDirectory, finalName, classifier); + MavenArchiver archiver = new MavenArchiver(); + archiver.setArchiver(jarArchiver); + archiver.setOutputFile(narFile); + archive.setForced(forceCreation); + + try { + File contentDirectory = getClassesDirectory(); + if (!contentDirectory.exists()) { + getLog().warn("NAR will be empty - no content was marked for inclusion!"); + } else { + archiver.getArchiver().addDirectory(contentDirectory, getIncludes(), getExcludes()); + } + + File existingManifest = defaultManifestFile; + if (useDefaultManifestFile && existingManifest.exists() && archive.getManifestFile() == null) { + getLog().info("Adding existing MANIFEST to archive. Found under: " + existingManifest.getPath()); + archive.setManifestFile(existingManifest); + } + + // automatically add the artifact id to the manifest + archive.addManifestEntry("Nar-Id", project.getArtifactId()); + + // look for a nar dependency + String narDependency = getNarDependency(); + if (narDependency != null) { + archive.addManifestEntry("Nar-Dependency-Id", narDependency); + } + + archiver.createArchive(session, project, archive); + return narFile; + } catch (ArchiverException | MojoExecutionException | ManifestException | IOException | DependencyResolutionRequiredException e) { + throw new MojoExecutionException("Error assembling NAR", e); + } + } + + private String[] getIncludes() { + if (includes != null && includes.length > 0) { + return includes; + } + return DEFAULT_INCLUDES; + } + + private String[] getExcludes() { + if (excludes != null && excludes.length > 0) { + return excludes; + } + return DEFAULT_EXCLUDES; + } + + protected File getNarFile(File basedir, String finalName, String classifier) { + if (classifier == null) { + classifier = ""; + } else if (classifier.trim().length() > 0 && !classifier.startsWith("-")) { + classifier = "-" + classifier; + } + + return new File(basedir, finalName + classifier + ".nar"); + } + + private String getNarDependency() throws MojoExecutionException { + String narDependency = null; + + // get nar dependencies + FilterArtifacts filter = new FilterArtifacts(); + filter.addFilter(new TypeFilter("nar", "")); + + // start with all artifacts. + Set artifacts = project.getArtifacts(); + + // perform filtering + try { + artifacts = filter.filter(artifacts); + } catch (ArtifactFilterException e) { + throw new MojoExecutionException(e.getMessage(), e); + } + + // ensure there is a single nar dependency + if (artifacts.size() > 1) { + throw new MojoExecutionException("Each NAR represents a ClassLoader. A NAR dependency allows that NAR's ClassLoader to be " + + "used as the parent of this NAR's ClassLoader. As a result, only a single NAR dependency is allowed."); + } else if (artifacts.size() == 1) { + final Artifact artifact = (Artifact) artifacts.iterator().next(); + narDependency = artifact.getArtifactId(); + } + + return narDependency; + } +} diff --git a/misc/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml b/misc/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml new file mode 100644 index 0000000000..0680d18e23 --- /dev/null +++ b/misc/nar-maven-plugin/src/main/resources/META-INF/plexus/components.xml @@ -0,0 +1,52 @@ + + + + + + org.apache.maven.lifecycle.mapping.LifecycleMapping + nar + org.apache.maven.lifecycle.mapping.DefaultLifecycleMapping + + + + default + + org.apache.maven.plugins:maven-resources-plugin:resources + org.apache.maven.plugins:maven-compiler-plugin:compile + org.apache.maven.plugins:maven-resources-plugin:testResources + org.apache.maven.plugins:maven-compiler-plugin:testCompile + org.apache.maven.plugins:maven-surefire-plugin:test + org.apache.nifi:nar-maven-plugin:nar + org.apache.maven.plugins:maven-install-plugin:install + org.apache.maven.plugins:maven-deploy-plugin:deploy + + + + + + + org.apache.maven.artifact.handler.ArtifactHandler + nar + org.apache.maven.artifact.handler.DefaultArtifactHandler + + nar + java + false + true + + + + diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml new file mode 100644 index 0000000000..6280349ae0 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/pom.xml @@ -0,0 +1,67 @@ + + + 4.0.0 + + + org.apache.nifi + distributed-cache-services-bundle + 0.0.1-SNAPSHOT + + + distributed-cache-client-service + jar + + Distributed Cache Client Service + Provides a Client for interfacing with a Distributed Cache + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + distributed-cache-client-service-api + + + org.apache.nifi + distributed-cache-protocol + ${project.version} + + + org.apache.nifi + remote-communications-utils + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-stream-utils + + + org.apache.nifi + ssl-context-service-api + + + junit + junit + 4.9 + test + + + diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.java new file mode 100644 index 0000000000..f838c2f0d7 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/CommsSession.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.distributed.cache.client; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLContext; + +public interface CommsSession extends Closeable { + + void setTimeout(final long value, final TimeUnit timeUnit); + + InputStream getInputStream() throws IOException; + + OutputStream getOutputStream() throws IOException; + + boolean isClosed(); + + void interrupt(); + + String getHostname(); + + int getPort(); + + long getTimeout(TimeUnit timeUnit); + + SSLContext getSSLContext(); +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java new file mode 100644 index 0000000000..ee96660a2d --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedMapCacheClientService.java @@ -0,0 +1,301 @@ +/* + * 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.distributed.cache.client; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.distributed.cache.protocol.ProtocolHandshake; +import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException; +import org.apache.nifi.io.ByteArrayOutputStream; +import org.apache.nifi.io.DataOutputStream; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.remote.StandardVersionNegotiator; +import org.apache.nifi.remote.VersionNegotiator; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.nifi.ssl.SSLContextService.ClientAuth; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DistributedMapCacheClientService extends AbstractControllerService implements DistributedMapCacheClient { + + private static final Logger logger = LoggerFactory.getLogger(DistributedMapCacheClientService.class); + + public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder() + .name("Server Hostname") + .description("The name of the server that is running the DistributedMapCacheServer service") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder() + .name("Server Port") + .description("The port on the remote server that is to be used when communicating with the DistributedMapCacheServer service") + .required(true) + .addValidator(StandardValidators.PORT_VALIDATOR) + .defaultValue("4557") + .build(); + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("SSL Context Service") + .description( + "If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted") + .required(false) + .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class)) + .defaultValue(null) + .build(); + public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder() + .name("Communications Timeout") + .description( + "Specifies how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("30 secs") + .build(); + + private final BlockingQueue queue = new LinkedBlockingQueue<>(); + private volatile ConfigurationContext configContext; + private volatile boolean closed = false; + + @Override + protected List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(HOSTNAME); + descriptors.add(PORT); + descriptors.add(SSL_CONTEXT_SERVICE); + descriptors.add(COMMUNICATIONS_TIMEOUT); + return descriptors; + } + + @OnConfigured + public void cacheConfig(final ConfigurationContext context) { + this.configContext = context; + } + + @Override + public boolean putIfAbsent(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer) + throws IOException { + return withCommsSession(new CommsAction() { + @Override + public Boolean execute(final CommsSession session) throws IOException { + final DataOutputStream dos = new DataOutputStream(session.getOutputStream()); + dos.writeUTF("putIfAbsent"); + + serialize(key, keySerializer, dos); + serialize(value, valueSerializer, dos); + + dos.flush(); + + final DataInputStream dis = new DataInputStream(session.getInputStream()); + return dis.readBoolean(); + } + }); + } + + @Override + public boolean containsKey(final K key, final Serializer keySerializer) throws IOException { + return withCommsSession(new CommsAction() { + @Override + public Boolean execute(final CommsSession session) throws IOException { + final DataOutputStream dos = new DataOutputStream(session.getOutputStream()); + dos.writeUTF("containsKey"); + + serialize(key, keySerializer, dos); + dos.flush(); + + final DataInputStream dis = new DataInputStream(session.getInputStream()); + return dis.readBoolean(); + } + }); + } + + @Override + public V getAndPutIfAbsent(final K key, final V value, final Serializer keySerializer, final Serializer valueSerializer, + final Deserializer valueDeserializer) throws IOException { + return withCommsSession(new CommsAction() { + @Override + public V execute(final CommsSession session) throws IOException { + final DataOutputStream dos = new DataOutputStream(session.getOutputStream()); + dos.writeUTF("getAndPutIfAbsent"); + + serialize(key, keySerializer, dos); + serialize(value, valueSerializer, dos); + dos.flush(); + + // read response + final DataInputStream dis = new DataInputStream(session.getInputStream()); + final byte[] responseBuffer = readLengthDelimitedResponse(dis); + return valueDeserializer.deserialize(responseBuffer); + } + }); + } + + @Override + public V get(final K key, final Serializer keySerializer, final Deserializer valueDeserializer) throws IOException { + return withCommsSession(new CommsAction() { + @Override + public V execute(final CommsSession session) throws IOException { + final DataOutputStream dos = new DataOutputStream(session.getOutputStream()); + dos.writeUTF("get"); + + serialize(key, keySerializer, dos); + dos.flush(); + + // read response + final DataInputStream dis = new DataInputStream(session.getInputStream()); + final byte[] responseBuffer = readLengthDelimitedResponse(dis); + return valueDeserializer.deserialize(responseBuffer); + } + }); + } + + @Override + public boolean remove(final K key, final Serializer serializer) throws IOException { + return withCommsSession(new CommsAction() { + @Override + public Boolean execute(final CommsSession session) throws IOException { + final DataOutputStream dos = new DataOutputStream(session.getOutputStream()); + dos.writeUTF("remove"); + + serialize(key, serializer, dos); + dos.flush(); + + // read response + final DataInputStream dis = new DataInputStream(session.getInputStream()); + return dis.readBoolean(); + } + }); + } + + private byte[] readLengthDelimitedResponse(final DataInputStream dis) throws IOException { + final int responseLength = dis.readInt(); + final byte[] responseBuffer = new byte[responseLength]; + dis.readFully(responseBuffer); + return responseBuffer; + } + + public CommsSession createCommsSession(final ConfigurationContext context) throws IOException { + final String hostname = context.getProperty(HOSTNAME).getValue(); + final int port = context.getProperty(PORT).asInteger(); + final long timeoutMillis = context.getProperty(COMMUNICATIONS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + + final CommsSession commsSession; + if (sslContextService == null) { + commsSession = new StandardCommsSession(hostname, port); + } else { + commsSession = new SSLCommsSession(sslContextService.createSSLContext(ClientAuth.REQUIRED), hostname, port); + } + + commsSession.setTimeout(timeoutMillis, TimeUnit.MILLISECONDS); + return commsSession; + } + + private CommsSession leaseCommsSession() throws IOException { + CommsSession session = queue.poll(); + if (session != null && !session.isClosed()) { + return session; + } + + session = createCommsSession(configContext); + final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1); + try { + ProtocolHandshake.initiateHandshake(session.getInputStream(), session.getOutputStream(), versionNegotiator); + } catch (final HandshakeException e) { + try { + session.close(); + } catch (final IOException ioe) { + } + + throw new IOException(e); + } + + return session; + } + + @Override + public void close() throws IOException { + this.closed = true; + + CommsSession commsSession; + while ((commsSession = queue.poll()) != null) { + try (final DataOutputStream dos = new DataOutputStream(commsSession.getOutputStream())) { + dos.writeUTF("close"); + dos.flush(); + commsSession.close(); + } catch (final IOException e) { + } + } + logger.info("Closed {}", new Object[] { getIdentifier() }); + } + + @Override + protected void finalize() throws Throwable { + if (!closed) + close(); + logger.debug("Finalize called"); + } + + private void serialize(final T value, final Serializer serializer, final DataOutputStream dos) throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.serialize(value, baos); + dos.writeInt(baos.size()); + baos.writeTo(dos); + } + + private T withCommsSession(final CommsAction action) throws IOException { + if (closed) { + throw new IllegalStateException("Client is closed"); + } + + final CommsSession session = leaseCommsSession(); + try { + return action.execute(session); + } catch (final IOException ioe) { + try { + session.close(); + } catch (final IOException ignored) { + } + + throw ioe; + } finally { + if (!session.isClosed()) { + if (this.closed) { + try { + session.close(); + } catch (final IOException ioe) { + } + } else { + queue.offer(session); + } + } + } + } + + private static interface CommsAction { + T execute(CommsSession commsSession) throws IOException; + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.java new file mode 100644 index 0000000000..1d7c94c2e5 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/DistributedSetCacheClientService.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.distributed.cache.client; + +import java.io.DataInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.distributed.cache.protocol.ProtocolHandshake; +import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException; +import org.apache.nifi.io.ByteArrayOutputStream; +import org.apache.nifi.io.DataOutputStream; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.remote.StandardVersionNegotiator; +import org.apache.nifi.remote.VersionNegotiator; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.nifi.ssl.SSLContextService.ClientAuth; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DistributedSetCacheClientService extends AbstractControllerService implements DistributedSetCacheClient { + + private static final Logger logger = LoggerFactory.getLogger(DistributedMapCacheClientService.class); + + public static final PropertyDescriptor HOSTNAME = new PropertyDescriptor.Builder() + .name("Server Hostname") + .description("The name of the server that is running the DistributedSetCacheServer service") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder() + .name("Server Port") + .description("The port on the remote server that is to be used when communicating with the DistributedSetCacheServer service") + .required(true) + .addValidator(StandardValidators.PORT_VALIDATOR) + .defaultValue("4557") + .build(); + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("SSL Context Service") + .description( + "If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted") + .required(false) + .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class)) + .defaultValue(null) + .build(); + public static final PropertyDescriptor COMMUNICATIONS_TIMEOUT = new PropertyDescriptor.Builder() + .name("Communications Timeout") + .description( + "Specifices how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received") + .required(true) + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .defaultValue("30 secs") + .build(); + + private final BlockingQueue queue = new LinkedBlockingQueue<>(); + private volatile ConfigurationContext configContext; + private volatile boolean closed = false; + + @Override + protected List getSupportedPropertyDescriptors() { + final List descriptors = new ArrayList<>(); + descriptors.add(HOSTNAME); + descriptors.add(PORT); + descriptors.add(SSL_CONTEXT_SERVICE); + descriptors.add(COMMUNICATIONS_TIMEOUT); + return descriptors; + } + + @OnConfigured + public void onConfigured(final ConfigurationContext context) { + this.configContext = context; + } + + public CommsSession createCommsSession(final ConfigurationContext context) throws IOException { + final String hostname = context.getProperty(HOSTNAME).getValue(); + final int port = context.getProperty(PORT).asInteger(); + final long timeoutMillis = context.getProperty(COMMUNICATIONS_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS); + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + + final CommsSession commsSession; + if (sslContextService == null) { + commsSession = new StandardCommsSession(hostname, port); + } else { + commsSession = new SSLCommsSession(sslContextService.createSSLContext(ClientAuth.REQUIRED), hostname, port); + } + + commsSession.setTimeout(timeoutMillis, TimeUnit.MILLISECONDS); + return commsSession; + } + + private CommsSession leaseCommsSession() throws IOException { + CommsSession session = queue.poll(); + if (session != null && !session.isClosed()) { + return session; + } + + session = createCommsSession(configContext); + final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1); + try { + ProtocolHandshake.initiateHandshake(session.getInputStream(), session.getOutputStream(), versionNegotiator); + } catch (final HandshakeException e) { + try { + session.close(); + } catch (final IOException ioe) { + } + + throw new IOException(e); + } + + return session; + } + + @Override + public boolean addIfAbsent(final T value, final Serializer serializer) throws IOException { + return invokeRemoteBoolean("addIfAbsent", value, serializer); + } + + @Override + public boolean contains(final T value, final Serializer serializer) throws IOException { + return invokeRemoteBoolean("contains", value, serializer); + } + + @Override + public boolean remove(final T value, final Serializer serializer) throws IOException { + return invokeRemoteBoolean("remove", value, serializer); + } + + @Override + public void close() throws IOException { + this.closed = true; + + CommsSession commsSession; + while ((commsSession = queue.poll()) != null) { + try (final DataOutputStream dos = new DataOutputStream(commsSession.getOutputStream())) { + dos.writeUTF("close"); + dos.flush(); + commsSession.close(); + } catch (final IOException e) { + } + } + logger.info("Closed {}", new Object[] { getIdentifier() }); + } + + @Override + protected void finalize() throws Throwable { + if (!closed) + close(); + logger.debug("Finalize called"); + } + + private boolean invokeRemoteBoolean(final String methodName, final T value, final Serializer serializer) throws IOException { + if (closed) { + throw new IllegalStateException("Client is closed"); + } + + final CommsSession session = leaseCommsSession(); + try { + final DataOutputStream dos = new DataOutputStream(session.getOutputStream()); + dos.writeUTF(methodName); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + serializer.serialize(value, baos); + dos.writeInt(baos.size()); + baos.writeTo(dos); + dos.flush(); + + final DataInputStream dis = new DataInputStream(session.getInputStream()); + return dis.readBoolean(); + } catch (final IOException ioe) { + try { + session.close(); + } catch (final IOException ignored) { + } + + throw ioe; + } finally { + if (!session.isClosed()) { + if (this.closed) { + try { + session.close(); + } catch (final IOException ioe) { + } + } else { + queue.offer(session); + } + } + } + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java new file mode 100644 index 0000000000..c8be082fac --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/SSLCommsSession.java @@ -0,0 +1,106 @@ +/* + * 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.distributed.cache.client; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.io.BufferedOutputStream; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelOutputStream; + +public class SSLCommsSession implements CommsSession { + private final SSLSocketChannel sslSocketChannel; + private final SSLContext sslContext; + private final String hostname; + private final int port; + + private final SSLSocketChannelInputStream in; + private final BufferedInputStream bufferedIn; + + private final SSLSocketChannelOutputStream out; + private final BufferedOutputStream bufferedOut; + + public SSLCommsSession(final SSLContext sslContext, final String hostname, final int port) throws IOException { + sslSocketChannel = new SSLSocketChannel(sslContext, hostname, port, true); + + in = new SSLSocketChannelInputStream(sslSocketChannel); + bufferedIn = new BufferedInputStream(in); + + out = new SSLSocketChannelOutputStream(sslSocketChannel); + bufferedOut = new BufferedOutputStream(out); + + this.sslContext = sslContext; + this.hostname = hostname; + this.port = port; + } + + @Override + public void interrupt() { + sslSocketChannel.interrupt(); + } + + @Override + public void close() throws IOException { + sslSocketChannel.close(); + } + + @Override + public void setTimeout(final long value, final TimeUnit timeUnit) { + sslSocketChannel.setTimeout((int) TimeUnit.MILLISECONDS.convert(value, timeUnit)); + } + + @Override + public InputStream getInputStream() throws IOException { + return bufferedIn; + } + + @Override + public OutputStream getOutputStream() throws IOException { + return bufferedOut; + } + + @Override + public boolean isClosed() { + return sslSocketChannel.isClosed(); + } + + @Override + public String getHostname() { + return hostname; + } + + @Override + public int getPort() { + return port; + } + @Override + public SSLContext getSSLContext() { + return sslContext; + } + @Override + public long getTimeout(final TimeUnit timeUnit) { + return timeUnit.convert(sslSocketChannel.getTimeout(), TimeUnit.MILLISECONDS); + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java new file mode 100644 index 0000000000..bbe2917e9f --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/java/org/apache/nifi/distributed/cache/client/StandardCommsSession.java @@ -0,0 +1,124 @@ +/* + * 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.distributed.cache.client; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.nio.channels.SocketChannel; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.io.BufferedOutputStream; +import org.apache.nifi.remote.io.InterruptableInputStream; +import org.apache.nifi.remote.io.InterruptableOutputStream; +import org.apache.nifi.remote.io.socket.SocketChannelInputStream; +import org.apache.nifi.remote.io.socket.SocketChannelOutputStream; + +public class StandardCommsSession implements CommsSession { + private final SocketChannel socketChannel; + private final String hostname; + private final int port; + private volatile long timeoutMillis; + + private final SocketChannelInputStream in; + private final InterruptableInputStream bufferedIn; + + private final SocketChannelOutputStream out; + private final InterruptableOutputStream bufferedOut; + + public StandardCommsSession(final String hostname, final int port) throws IOException { + socketChannel = SocketChannel.open(new InetSocketAddress(hostname, port)); + socketChannel.configureBlocking(false); + in = new SocketChannelInputStream(socketChannel); + bufferedIn = new InterruptableInputStream(new BufferedInputStream(in)); + + out = new SocketChannelOutputStream(socketChannel); + bufferedOut = new InterruptableOutputStream(new BufferedOutputStream(out)); + + this.hostname = hostname; + this.port = port; + } + + @Override + public void interrupt() { + bufferedIn.interrupt(); + bufferedOut.interrupt(); + } + + @Override + public void close() throws IOException { + socketChannel.close(); + } + + @Override + public void setTimeout(final long value, final TimeUnit timeUnit) { + in.setTimeout((int) TimeUnit.MILLISECONDS.convert(value, timeUnit)); + out.setTimeout((int) TimeUnit.MILLISECONDS.convert(value, timeUnit)); + timeoutMillis = TimeUnit.MILLISECONDS.convert(value, timeUnit); + } + + @Override + public InputStream getInputStream() throws IOException { + return bufferedIn; + } + + @Override + public OutputStream getOutputStream() throws IOException { + return bufferedOut; + } + + @Override + public boolean isClosed() { + boolean closed = !socketChannel.isConnected(); + if (!closed) { + try { + this.in.isDataAvailable(); + } catch (IOException e) { + try { + close(); + } catch (IOException e1) { + } + closed = true; + } + } + return closed; + } + + @Override + public String getHostname() { + return hostname; + } + + @Override + public int getPort() { + return port; + } + + @Override + public SSLContext getSSLContext() { + return null; + } + + @Override + public long getTimeout(final TimeUnit timeUnit) { + return timeUnit.convert(timeoutMillis, TimeUnit.MILLISECONDS); + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 0000000000..a91f7ee4f4 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService +org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService \ No newline at end of file diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html new file mode 100644 index 0000000000..d5f359567e --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService/index.html @@ -0,0 +1,78 @@ + + + + + +Distributed Map Cache Client Service + + + + +

Description:

+ +

A Controller Service that can be used to communicate with a + Distributed Map Cache Server.

+ + + +

+ Properties: +

+

In the list below, the names of required properties appear + in bold. Any other properties (not in bold) are considered optional. + If a property has a default value, it is indicated. If a property + supports the use of the NiFi Expression Language (or simply, + "expression language"), that is also indicated.

+ +
    +
  • Server Hostname +
      +
    • The name of the server that is running the DistributedMapCacheServer service
    • +
    • Default value: no default
    • +
    • Supports expression language: false
    • +
  • +
  • Server Port +
      +
    • The port on the remote server that is to be used when communicating with the + DistributedMapCacheServer service
    • + +
    • Default value: 4557
    • +
    • Supports expression language: false
    • +
  • +
  • SSL Context Service +
      +
    • If specified, indicates the SSL Context Service that is used to communicate with the remote server. If not specified, communications will not be encrypted +
    • Default value: no default
    • +
    • Supports expression language: false
    • +
  • +
  • Communications Timeout +
      +
    • Specifices how long to wait when communicating with the remote server before determining that there is a communications failure if data cannot be sent or received +
    • Default value: 30 secs
    • +
    • Supports expression language: false
    • +
  • + +
+ + + See Also: + + + + diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html b/nar-bundles/distributed-cache-services-bundle/distributed-cache-client-service/src/main/resources/docs/org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService/index.html new file mode 100755 index 0000000000..e69de29bb2 diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml new file mode 100644 index 0000000000..bc612aeb86 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/pom.xml @@ -0,0 +1,39 @@ + + + 4.0.0 + + org.apache.nifi + distributed-cache-services-bundle + 0.0.1-SNAPSHOT + + + distributed-cache-protocol + Distributed Cache Protocol + + + Defines the communications protocol that is used between clients and servers + for the Distributed Cache services + + + + + + org.apache.nifi + remote-communications-utils + + + + \ No newline at end of file diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java new file mode 100644 index 0000000000..da2acade49 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/ProtocolHandshake.java @@ -0,0 +1,119 @@ +/* + * 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.distributed.cache.protocol; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException; +import org.apache.nifi.remote.VersionNegotiator; + +public class ProtocolHandshake { + + public static final byte[] MAGIC_HEADER = new byte[] { 'N', 'i', 'F', 'i' }; + + public static final int RESOURCE_OK = 20; + public static final int DIFFERENT_RESOURCE_VERSION = 21; + public static final int ABORT = 255; + + + public static void initiateHandshake(final InputStream in, final OutputStream out, final VersionNegotiator versionNegotiator) throws IOException, HandshakeException { + final DataInputStream dis = new DataInputStream(in); + final DataOutputStream dos = new DataOutputStream(out); + + try { + dos.write(MAGIC_HEADER); + + initiateVersionNegotiation(versionNegotiator, dis, dos); + } finally { + dos.flush(); + } + } + + + public static void receiveHandshake(final InputStream in, final OutputStream out, final VersionNegotiator versionNegotiator) throws IOException, HandshakeException { + final DataInputStream dis = new DataInputStream(in); + final DataOutputStream dos = new DataOutputStream(out); + + try { + final byte[] magicHeaderBuffer = new byte[4]; + dis.readFully(magicHeaderBuffer); + + receiveVersionNegotiation(versionNegotiator, dis, dos); + } finally { + dos.flush(); + } + } + + + private static void initiateVersionNegotiation(final VersionNegotiator negotiator, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException { + // Write the classname of the RemoteStreamCodec, followed by its version + dos.writeInt(negotiator.getVersion()); + dos.flush(); + + // wait for response from server. + final int statusCode = dis.read(); + switch (statusCode) { + case RESOURCE_OK: // server accepted our proposal of codec name/version + return; + case DIFFERENT_RESOURCE_VERSION: // server accepted our proposal of codec name but not the version + // Get server's preferred version + final int newVersion = dis.readInt(); + + // Determine our new preferred version that is no greater than the server's preferred version. + final Integer newPreference = negotiator.getPreferredVersion(newVersion); + // If we could not agree with server on a version, fail now. + if ( newPreference == null ) { + throw new HandshakeException("Could not agree on protocol version"); + } + + negotiator.setVersion(newPreference); + + // Attempt negotiation of resource based on our new preferred version. + initiateVersionNegotiation(negotiator, dis, dos); + case ABORT: + throw new HandshakeException("Remote destination aborted connection with message: " + dis.readUTF()); + default: + throw new HandshakeException("Received unexpected response code " + statusCode + " when negotiating version with remote server"); + } + } + + private static void receiveVersionNegotiation(final VersionNegotiator negotiator, final DataInputStream dis, final DataOutputStream dos) throws IOException, HandshakeException { + final int version = dis.readInt(); + if ( negotiator.isVersionSupported(version) ) { + dos.write(RESOURCE_OK); + dos.flush(); + + negotiator.setVersion(version); + } else { + final Integer preferred = negotiator.getPreferredVersion(version); + if ( preferred == null ) { + dos.write(ABORT); + dos.flush(); + throw new HandshakeException("Unable to negotiate an acceptable version of the Distributed Cache Protocol"); + } + dos.write(DIFFERENT_RESOURCE_VERSION); + dos.writeInt(preferred); + dos.flush(); + + receiveVersionNegotiation(negotiator, dis, dos); + } + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.java new file mode 100644 index 0000000000..8049d42b20 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-protocol/src/main/java/org/apache/nifi/distributed/cache/protocol/exception/HandshakeException.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.distributed.cache.protocol.exception; + +public class HandshakeException extends Exception { + public HandshakeException(final String message) { + super(message); + } + + public HandshakeException(final Throwable cause) { + super(cause); + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml new file mode 100644 index 0000000000..5dec322595 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/pom.xml @@ -0,0 +1,81 @@ + + + 4.0.0 + + + org.apache.nifi + distributed-cache-services-bundle + 0.0.1-SNAPSHOT + + + distributed-cache-server + + Distributed Cache Server + Provides a Controller Service for hosting Distributed Caches + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + distributed-cache-protocol + ${project.version} + + + org.apache.nifi + remote-communications-utils + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-stream-utils + + + org.apache.nifi + ssl-context-service-api + + + wali + wali + + + org.apache.nifi + distributed-cache-client-service-api + test + + + org.apache.nifi + distributed-cache-client-service + ${project.version} + test + + + org.apache.nifi + nifi-mock + + + + org.apache.nifi + ssl-context-service + + + + \ No newline at end of file diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java new file mode 100644 index 0000000000..9b4e70ef3b --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/AbstractCacheServer.java @@ -0,0 +1,199 @@ +/* + * 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.distributed.cache.server; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.SocketTimeoutException; +import java.nio.channels.ServerSocketChannel; +import java.nio.channels.SocketChannel; +import java.util.Set; +import java.util.concurrent.CopyOnWriteArraySet; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.distributed.cache.protocol.ProtocolHandshake; +import org.apache.nifi.distributed.cache.protocol.exception.HandshakeException; +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.io.BufferedOutputStream; +import org.apache.nifi.remote.StandardVersionNegotiator; +import org.apache.nifi.remote.VersionNegotiator; +import org.apache.nifi.remote.io.socket.SocketChannelInputStream; +import org.apache.nifi.remote.io.socket.SocketChannelOutputStream; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannel; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelInputStream; +import org.apache.nifi.remote.io.socket.ssl.SSLSocketChannelOutputStream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class AbstractCacheServer implements CacheServer { + + private static final Logger logger = LoggerFactory.getLogger(AbstractCacheServer.class); + + private final String identifier; + private final int port; + private final SSLContext sslContext; + protected volatile boolean stopped = false; + private final Set processInputThreads = new CopyOnWriteArraySet<>();; + + private volatile ServerSocketChannel serverSocketChannel; + + public AbstractCacheServer(final String identifier, final SSLContext sslContext, final int port) { + this.identifier = identifier; + this.port = port; + this.sslContext = sslContext; + } + + @Override + public void start() throws IOException { + serverSocketChannel = ServerSocketChannel.open(); + serverSocketChannel.configureBlocking(true); + serverSocketChannel.bind(new InetSocketAddress(port)); + + final Runnable runnable = new Runnable() { + + @Override + public void run() { + while (true) { + final SocketChannel socketChannel; + try { + socketChannel = serverSocketChannel.accept(); + logger.debug("Connected to {}", new Object[] { socketChannel }); + } catch (final IOException e) { + if (!stopped) { + logger.error("{} unable to accept connection from remote peer due to {}", this, e.toString()); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + } + return; + } + + final Runnable processInputRunnable = new Runnable() { + @Override + public void run() { + final InputStream rawInputStream; + final OutputStream rawOutputStream; + final String peer = socketChannel.socket().getInetAddress().getHostName(); + + try { + if (sslContext == null) { + rawInputStream = new SocketChannelInputStream(socketChannel); + rawOutputStream = new SocketChannelOutputStream(socketChannel); + } else { + final SSLSocketChannel sslSocketChannel = new SSLSocketChannel(sslContext, socketChannel, false); + sslSocketChannel.connect(); + rawInputStream = new SSLSocketChannelInputStream(sslSocketChannel); + rawOutputStream = new SSLSocketChannelOutputStream(sslSocketChannel); + } + } catch (IOException e) { + logger.error("Cannot create input and/or output streams for {}", new Object[] { identifier }, e); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + try { + socketChannel.close(); + } catch (IOException swallow) { + } + + return; + } + try (final InputStream in = new BufferedInputStream(rawInputStream); + final OutputStream out = new BufferedOutputStream(rawOutputStream)) { + + final VersionNegotiator versionNegotiator = new StandardVersionNegotiator(1); + + ProtocolHandshake.receiveHandshake(in, out, versionNegotiator); + + boolean continueComms = true; + while (continueComms) { + continueComms = listen(in, out, versionNegotiator.getVersion()); + } + // client has issued 'close' + logger.debug("Client issued close on {}", new Object[] { socketChannel }); + } catch (final SocketTimeoutException e) { + logger.debug("30 sec timeout reached", e); + } catch (final IOException | HandshakeException e) { + if (!stopped) { + logger.error("{} unable to communicate with remote peer {} due to {}", new Object[] { this, peer, e.toString() }); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + } + } finally { + processInputThreads.remove(Thread.currentThread()); + } + } + }; + + final Thread processInputThread = new Thread(processInputRunnable); + processInputThread.setName("Distributed Cache Server Communications Thread: " + identifier); + processInputThread.setDaemon(true); + processInputThread.start(); + processInputThreads.add(processInputThread); + } + } + }; + + final Thread thread = new Thread(runnable); + thread.setDaemon(true); + thread.setName("Distributed Cache Server: " + identifier); + thread.start(); + } + + @Override + public void stop() throws IOException { + stopped = true; + logger.info("Stopping CacheServer {}", new Object[] { this.identifier }); + + if (serverSocketChannel != null) { + serverSocketChannel.close(); + } + // need to close out the created SocketChannels...this is done by interrupting + // the created threads that loop on listen(). + for (Thread processInputThread : processInputThreads) { + processInputThread.interrupt(); + int i = 0; + while (!processInputThread.isInterrupted() && i++ < 5) { + try { + Thread.sleep(50); // allow thread to gracefully terminate + } catch (InterruptedException e) { + } + } + } + processInputThreads.clear(); + } + + @Override + public String toString() { + return "CacheServer[id=" + identifier + "]"; + } + + /** + * Listens for incoming data and communicates with remote peer + * + * @param in + * @param out + * @param version + * @return true if communications should continue, false otherwise + * @throws IOException + */ + protected abstract boolean listen(InputStream in, OutputStream out, int version) throws IOException; +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.java new file mode 100644 index 0000000000..71ac56d0d2 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheRecord.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.distributed.cache.server; + +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; + +public class CacheRecord { + + private static final AtomicLong idGenerator = new AtomicLong(0L); + + private final long id; + private final long entryDate; + private volatile long lastHitDate; + private final AtomicInteger hitCount = new AtomicInteger(0); + + public CacheRecord() { + entryDate = System.currentTimeMillis(); + lastHitDate = entryDate; + id = idGenerator.getAndIncrement(); + } + + public long getEntryDate() { + return entryDate; + } + + public long getLastHitDate() { + return lastHitDate; + } + + public int getHitCount() { + return hitCount.get(); + } + + public void hit() { + hitCount.getAndIncrement(); + lastHitDate = System.currentTimeMillis(); + } + + public long getId() { + return id; + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.java new file mode 100644 index 0000000000..2c85cd85da --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/CacheServer.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.distributed.cache.server; + +import java.io.IOException; + +public interface CacheServer { + + void start() throws IOException; + void stop() throws IOException; + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java new file mode 100644 index 0000000000..0f962d0249 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedCacheServer.java @@ -0,0 +1,107 @@ +/* + * 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.distributed.cache.server; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.annotation.OnConfigured; +import org.apache.nifi.processor.annotation.OnShutdown; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.ssl.SSLContextService; + +public abstract class DistributedCacheServer extends AbstractControllerService { + public static final String EVICTION_STRATEGY_LFU = "Least Frequently Used"; + public static final String EVICTION_STRATEGY_LRU = "Least Recently Used"; + public static final String EVICTION_STRATEGY_FIFO = "First In, First Out"; + + public static final PropertyDescriptor PORT = new PropertyDescriptor.Builder() + .name("Port") + .description("The port to listen on for incoming connections") + .required(true) + .addValidator(StandardValidators.PORT_VALIDATOR) + .defaultValue("4557") + .build(); + public static final PropertyDescriptor SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder() + .name("SSL Context Service") + .description( + "If specified, this service will be used to create an SSL Context that will be used to secure communications; if not specified, communications will not be secure") + .required(false) + .addValidator(StandardValidators.createControllerServiceExistsValidator(SSLContextService.class)) + .build(); + public static final PropertyDescriptor MAX_CACHE_ENTRIES = new PropertyDescriptor.Builder() + .name("Maximum Cache Entries") + .description("The maximum number of cache entries that the cache can hold") + .required(true) + .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) + .defaultValue("10000") + .build(); + public static final PropertyDescriptor EVICTION_POLICY = new PropertyDescriptor.Builder() + .name("Eviction Strategy") + .description("Determines which strategy should be used to evict values from the cache to make room for new entries") + .required(true) + .allowableValues(EVICTION_STRATEGY_LFU, EVICTION_STRATEGY_LRU, EVICTION_STRATEGY_FIFO) + .defaultValue(EVICTION_STRATEGY_LFU) + .build(); + public static final PropertyDescriptor PERSISTENCE_PATH = new PropertyDescriptor.Builder() + .name("Persistence Directory") + .description("If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only") + .required(false) + .addValidator(StandardValidators.createDirectoryExistsValidator(true, true)) + .build(); + + private volatile CacheServer cacheServer; + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(); + properties.add(PORT); + properties.add(MAX_CACHE_ENTRIES); + properties.add(EVICTION_POLICY); + properties.add(PERSISTENCE_PATH); + properties.add(new PropertyDescriptor.Builder().fromPropertyDescriptor(SSL_CONTEXT_SERVICE).allowableValues( + getControllerServiceLookup().getControllerServiceIdentifiers(SSLContextService.class)).build()); + return properties; + } + + @OnConfigured + public void startServer(final ConfigurationContext context) throws IOException { + if (cacheServer == null) { + cacheServer = createCacheServer(context); + cacheServer.start(); + } + } + + @OnShutdown + public void shutdownServer() throws IOException { + if (cacheServer != null) { + cacheServer.stop(); + } + cacheServer = null; + } + + @Override + protected void finalize() throws Throwable { + shutdownServer(); + } + + protected abstract CacheServer createCacheServer(ConfigurationContext context); +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java new file mode 100644 index 0000000000..426573fdaf --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/DistributedSetCacheServer.java @@ -0,0 +1,68 @@ +/* + * 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.distributed.cache.server; + +import java.io.File; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.nifi.ssl.SSLContextService.ClientAuth; + +public class DistributedSetCacheServer extends DistributedCacheServer { + + @Override + protected CacheServer createCacheServer(final ConfigurationContext context) { + final int port = context.getProperty(PORT).asInteger(); + final String persistencePath = context.getProperty(PERSISTENCE_PATH).getValue(); + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger(); + final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue(); + + final SSLContext sslContext; + if ( sslContextService == null ) { + sslContext = null; + } else { + sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED); + } + + final EvictionPolicy evictionPolicy; + switch (evictionPolicyName) { + case EVICTION_STRATEGY_FIFO: + evictionPolicy = EvictionPolicy.FIFO; + break; + case EVICTION_STRATEGY_LFU: + evictionPolicy = EvictionPolicy.LFU; + break; + case EVICTION_STRATEGY_LRU: + evictionPolicy = EvictionPolicy.LRU; + break; + default: + throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName); + } + + try { + final File persistenceDir = persistencePath == null ? null : new File(persistencePath); + + return new SetCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java new file mode 100644 index 0000000000..60bd2c19fd --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/EvictionPolicy.java @@ -0,0 +1,73 @@ +/* + * 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.distributed.cache.server; + +import java.util.Comparator; + +public enum EvictionPolicy { + LFU(new LFUComparator()), + LRU(new LRUComparator()), + FIFO(new FIFOComparator()); + + private final Comparator comparator; + + private EvictionPolicy(final Comparator comparator) { + this.comparator = comparator; + } + + public Comparator getComparator() { + return comparator; + } + + public static class LFUComparator implements Comparator { + @Override + public int compare(final CacheRecord o1, final CacheRecord o2) { + if ( o1.equals(o2) ) { + return 0; + } + + final int hitCountComparison = Integer.compare(o1.getHitCount(), o2.getHitCount()); + final int entryDateComparison = (hitCountComparison == 0) ? Long.compare(o1.getEntryDate(), o2.getEntryDate()) : hitCountComparison; + return (entryDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : entryDateComparison); + } + } + + public static class LRUComparator implements Comparator { + @Override + public int compare(final CacheRecord o1, final CacheRecord o2) { + if ( o1.equals(o2) ) { + return 0; + } + + final int lastHitDateComparison = Long.compare(o1.getLastHitDate(), o2.getLastHitDate()); + return (lastHitDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : lastHitDateComparison); + } + } + + public static class FIFOComparator implements Comparator { + @Override + public int compare(final CacheRecord o1, final CacheRecord o2) { + if ( o1.equals(o2) ) { + return 0; + } + + final int entryDateComparison = Long.compare(o1.getEntryDate(), o2.getEntryDate()); + return (entryDateComparison == 0 ? Long.compare(o1.getId(), o2.getId()) : entryDateComparison); + } + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java new file mode 100644 index 0000000000..5d2c0f6248 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/SetCacheServer.java @@ -0,0 +1,104 @@ +/* + * 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.distributed.cache.server; + +import java.io.DataInputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.distributed.cache.server.set.PersistentSetCache; +import org.apache.nifi.distributed.cache.server.set.SetCache; +import org.apache.nifi.distributed.cache.server.set.SetCacheResult; +import org.apache.nifi.distributed.cache.server.set.SimpleSetCache; +import org.apache.nifi.io.DataOutputStream; + +public class SetCacheServer extends AbstractCacheServer { + + private final SetCache cache; + + public SetCacheServer(final String identifier, final SSLContext sslContext, final int port, final int maxSize, + final EvictionPolicy evictionPolicy, final File persistencePath) throws IOException { + super(identifier, sslContext, port); + + final SetCache simpleCache = new SimpleSetCache(identifier, maxSize, evictionPolicy); + + if (persistencePath == null) { + this.cache = simpleCache; + } else { + final PersistentSetCache persistentCache = new PersistentSetCache(identifier, persistencePath, simpleCache); + persistentCache.restore(); + this.cache = persistentCache; + } + } + + @Override + protected boolean listen(final InputStream in, final OutputStream out, final int version) throws IOException { + final DataInputStream dis = new DataInputStream(in); + final DataOutputStream dos = new DataOutputStream(out); + + final String action = dis.readUTF(); + if (action.equals("close")) { + return false; + } + + final int valueLength = dis.readInt(); + final byte[] value = new byte[valueLength]; + dis.readFully(value); + final ByteBuffer valueBuffer = ByteBuffer.wrap(value); + + final SetCacheResult response; + switch (action) { + case "addIfAbsent": + response = cache.addIfAbsent(valueBuffer); + break; + case "contains": + response = cache.contains(valueBuffer); + break; + case "remove": + response = cache.remove(valueBuffer); + break; + default: + throw new IOException("IllegalRequest"); + } + + dos.writeBoolean(response.getResult()); + dos.flush(); + + return true; + } + + @Override + public void stop() throws IOException { + try { + super.stop(); + } finally { + cache.shutdown(); + } + } + + @Override + protected void finalize() throws Throwable { + if (!stopped) + stop(); + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java new file mode 100644 index 0000000000..920529de83 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/DistributedMapCacheServer.java @@ -0,0 +1,71 @@ +/* + * 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.distributed.cache.server.map; + +import java.io.File; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.distributed.cache.server.CacheServer; +import org.apache.nifi.distributed.cache.server.DistributedCacheServer; +import org.apache.nifi.distributed.cache.server.EvictionPolicy; +import org.apache.nifi.ssl.SSLContextService; +import org.apache.nifi.ssl.SSLContextService.ClientAuth; + +public class DistributedMapCacheServer extends DistributedCacheServer { + + @Override + protected CacheServer createCacheServer(final ConfigurationContext context) { + final int port = context.getProperty(PORT).asInteger(); + final String persistencePath = context.getProperty(PERSISTENCE_PATH).getValue(); + final SSLContextService sslContextService = context.getProperty(SSL_CONTEXT_SERVICE).asControllerService(SSLContextService.class); + final int maxSize = context.getProperty(MAX_CACHE_ENTRIES).asInteger(); + final String evictionPolicyName = context.getProperty(EVICTION_POLICY).getValue(); + + final SSLContext sslContext; + if ( sslContextService == null ) { + sslContext = null; + } else { + sslContext = sslContextService.createSSLContext(ClientAuth.REQUIRED); + } + + final EvictionPolicy evictionPolicy; + switch (evictionPolicyName) { + case EVICTION_STRATEGY_FIFO: + evictionPolicy = EvictionPolicy.FIFO; + break; + case EVICTION_STRATEGY_LFU: + evictionPolicy = EvictionPolicy.LFU; + break; + case EVICTION_STRATEGY_LRU: + evictionPolicy = EvictionPolicy.LRU; + break; + default: + throw new IllegalArgumentException("Illegal Eviction Policy: " + evictionPolicyName); + } + + try { + final File persistenceDir = persistencePath == null ? null : new File(persistencePath); + + return new MapCacheServer(getIdentifier(), sslContext, port, maxSize, evictionPolicy, persistenceDir); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java new file mode 100644 index 0000000000..534cb0b204 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCache.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.distributed.cache.server.map; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public interface MapCache { + + MapPutResult putIfAbsent(ByteBuffer key, ByteBuffer value) throws IOException; + boolean containsKey(ByteBuffer key) throws IOException; + ByteBuffer get(ByteBuffer key) throws IOException; + ByteBuffer remove(ByteBuffer key) throws IOException; + void shutdown() throws IOException; +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java new file mode 100644 index 0000000000..b0ab0c4003 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheRecord.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.distributed.cache.server.map; + +import java.nio.ByteBuffer; + +import org.apache.nifi.distributed.cache.server.CacheRecord; + +public class MapCacheRecord extends CacheRecord { + private final ByteBuffer key; + private final ByteBuffer value; + + public MapCacheRecord(final ByteBuffer key, final ByteBuffer value) { + this.key = key; + this.value = value; + } + + public ByteBuffer getKey() { + return key; + } + + public ByteBuffer getValue() { + return value; + } + + @Override + public int hashCode() { + return 2938476 + key.hashCode() * value.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if ( obj == this ) { + return true; + } + + if ( obj instanceof MapCacheRecord ) { + final MapCacheRecord that = ((MapCacheRecord) obj); + return key.equals(that.key) && value.equals(that.value); + } + + return false; + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java new file mode 100644 index 0000000000..3e8dd0e248 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapCacheServer.java @@ -0,0 +1,145 @@ +/* + * 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.distributed.cache.server.map; + +import java.io.DataInputStream; +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import javax.net.ssl.SSLContext; + +import org.apache.nifi.distributed.cache.server.AbstractCacheServer; +import org.apache.nifi.distributed.cache.server.EvictionPolicy; +import org.apache.nifi.io.DataOutputStream; + +public class MapCacheServer extends AbstractCacheServer { + + private final MapCache cache; + + public MapCacheServer(final String identifier, final SSLContext sslContext, final int port, final int maxSize, + final EvictionPolicy evictionPolicy, final File persistencePath) throws IOException { + super(identifier, sslContext, port); + + final MapCache simpleCache = new SimpleMapCache(identifier, maxSize, evictionPolicy); + + if (persistencePath == null) { + this.cache = simpleCache; + } else { + final PersistentMapCache persistentCache = new PersistentMapCache(identifier, persistencePath, simpleCache); + persistentCache.restore(); + this.cache = persistentCache; + } + } + + @Override + protected boolean listen(final InputStream in, final OutputStream out, final int version) throws IOException { + final DataInputStream dis = new DataInputStream(in); + final DataOutputStream dos = new DataOutputStream(out); + final String action = dis.readUTF(); + try { + switch (action) { + case "close": { + return false; + } + case "putIfAbsent": { + final byte[] key = readValue(dis); + final byte[] value = readValue(dis); + final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value)); + dos.writeBoolean(putResult.isSuccessful()); + break; + } + case "containsKey": { + final byte[] key = readValue(dis); + final boolean contains = cache.containsKey(ByteBuffer.wrap(key)); + dos.writeBoolean(contains); + break; + } + case "getAndPutIfAbsent": { + final byte[] key = readValue(dis); + final byte[] value = readValue(dis); + + final MapPutResult putResult = cache.putIfAbsent(ByteBuffer.wrap(key), ByteBuffer.wrap(value)); + if (putResult.isSuccessful()) { + // Put was successful. There was no old value to get. + dos.writeInt(0); + } else { + // we didn't put. Write back the previous value + final byte[] byteArray = putResult.getExistingValue().array(); + dos.writeInt(byteArray.length); + dos.write(byteArray); + } + + break; + } + case "get": { + final byte[] key = readValue(dis); + final ByteBuffer existingValue = cache.get(ByteBuffer.wrap(key)); + if (existingValue == null) { + // there was no existing value; we did a "put". + dos.writeInt(0); + } else { + // a value already existed. we did not update the map + final byte[] byteArray = existingValue.array(); + dos.writeInt(byteArray.length); + dos.write(byteArray); + } + + break; + } + case "remove": { + final byte[] key = readValue(dis); + final boolean removed = cache.remove(ByteBuffer.wrap(key)) != null; + dos.writeBoolean(removed); + break; + } + default: { + throw new IOException("Illegal Request"); + } + } + } finally { + dos.flush(); + } + + return true; + } + + @Override + public void stop() throws IOException { + try { + super.stop(); + } finally { + cache.shutdown(); + } + } + + @Override + protected void finalize() throws Throwable { + if (!stopped) + stop(); + } + + private byte[] readValue(final DataInputStream dis) throws IOException { + final int numBytes = dis.readInt(); + final byte[] buffer = new byte[numBytes]; + dis.readFully(buffer); + return buffer; + } + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java new file mode 100644 index 0000000000..29695eb32a --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/MapPutResult.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.distributed.cache.server.map; + +import java.nio.ByteBuffer; + +public class MapPutResult { + private final boolean successful; + private final ByteBuffer key, value; + private final ByteBuffer existingValue; + private final ByteBuffer evictedKey, evictedValue; + + public MapPutResult(final boolean successful, final ByteBuffer key, final ByteBuffer value, final ByteBuffer existingValue, final ByteBuffer evictedKey, final ByteBuffer evictedValue) { + this.successful = successful; + this.key = key; + this.value = value; + this.existingValue = existingValue; + this.evictedKey = evictedKey; + this.evictedValue = evictedValue; + } + + public boolean isSuccessful() { + return successful; + } + + public ByteBuffer getKey() { + return key; + } + + public ByteBuffer getValue() { + return value; + } + + public ByteBuffer getExistingValue() { + return existingValue; + } + + public ByteBuffer getEvictedKey() { + return evictedKey; + } + + public ByteBuffer getEvictedValue() { + return evictedValue; + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java new file mode 100644 index 0000000000..77fb77db0c --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/PersistentMapCache.java @@ -0,0 +1,210 @@ +/* + * 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.distributed.cache.server.map; + +import java.io.DataInputStream; +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.wali.MinimalLockingWriteAheadLog; +import org.wali.SerDe; +import org.wali.UpdateType; +import org.wali.WriteAheadRepository; + +public class PersistentMapCache implements MapCache { + + private final MapCache wrapped; + private final WriteAheadRepository wali; + + private final AtomicLong modifications = new AtomicLong(0L); + + public PersistentMapCache(final String serviceIdentifier, final File persistencePath, final MapCache cacheToWrap) throws IOException { + wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null); + wrapped = cacheToWrap; + } + + synchronized void restore() throws IOException { + final Collection recovered = wali.recoverRecords(); + for ( final MapWaliRecord record : recovered ) { + if ( record.getUpdateType() == UpdateType.CREATE ) { + wrapped.putIfAbsent(record.getKey(), record.getValue()); + } + } + } + + @Override + public MapPutResult putIfAbsent(final ByteBuffer key, final ByteBuffer value) throws IOException { + final MapPutResult putResult = wrapped.putIfAbsent(key, value); + if ( putResult.isSuccessful() ) { + // The put was successful. + final MapWaliRecord record = new MapWaliRecord(UpdateType.CREATE, key, value); + final List records = new ArrayList<>(); + records.add(record); + + if ( putResult.getEvictedKey() != null ) { + records.add(new MapWaliRecord(UpdateType.DELETE, putResult.getEvictedKey(), putResult.getEvictedValue())); + } + + wali.update(Collections.singletonList(record), false); + + final long modCount = modifications.getAndIncrement(); + if ( modCount > 0 && modCount % 100000 == 0 ) { + wali.checkpoint(); + } + } + + return putResult; + } + + @Override + public boolean containsKey(final ByteBuffer key) throws IOException { + return wrapped.containsKey(key); + } + + @Override + public ByteBuffer get(final ByteBuffer key) throws IOException { + return wrapped.get(key); + } + + @Override + public ByteBuffer remove(ByteBuffer key) throws IOException { + final ByteBuffer removeResult = wrapped.remove(key); + if ( removeResult != null ) { + final MapWaliRecord record = new MapWaliRecord(UpdateType.DELETE, key, removeResult); + final List records = new ArrayList<>(1); + records.add(record); + wali.update(records, false); + + final long modCount = modifications.getAndIncrement(); + if ( modCount > 0 && modCount % 1000 == 0 ) { + wali.checkpoint(); + } + } + return removeResult; + } + + + @Override + public void shutdown() throws IOException { + wali.shutdown(); + } + + + private static class MapWaliRecord { + private final UpdateType updateType; + private final ByteBuffer key; + private final ByteBuffer value; + + public MapWaliRecord(final UpdateType updateType, final ByteBuffer key, final ByteBuffer value) { + this.updateType = updateType; + this.key = key; + this.value = value; + } + + public UpdateType getUpdateType() { + return updateType; + } + + public ByteBuffer getKey() { + return key; + } + + public ByteBuffer getValue() { + return value; + } + } + + private static class Serde implements SerDe { + + @Override + public void serializeEdit(MapWaliRecord previousRecordState, MapWaliRecord newRecordState, java.io.DataOutputStream out) throws IOException { + final UpdateType updateType = newRecordState.getUpdateType(); + if ( updateType == UpdateType.DELETE ) { + out.write(0); + } else { + out.write(1); + } + + final byte[] key = newRecordState.getKey().array(); + final byte[] value = newRecordState.getValue().array(); + + out.writeInt(key.length); + out.write(key); + out.writeInt(value.length); + out.write(value); + } + + @Override + public void serializeRecord(MapWaliRecord record, java.io.DataOutputStream out) throws IOException { + serializeEdit(null, record, out); + } + + @Override + public MapWaliRecord deserializeEdit(final DataInputStream in, final Map currentRecordStates, final int version) throws IOException { + final int updateTypeValue = in.read(); + if ( updateTypeValue < 0 ) { + throw new EOFException(); + } + + final UpdateType updateType = (updateTypeValue == 0 ? UpdateType.DELETE : UpdateType.CREATE); + + final int keySize = in.readInt(); + final byte[] key = new byte[keySize]; + in.readFully(key); + + final int valueSize = in.readInt(); + final byte[] value = new byte[valueSize]; + in.readFully(value); + + return new MapWaliRecord(updateType, ByteBuffer.wrap(key), ByteBuffer.wrap(value)); + } + + @Override + public MapWaliRecord deserializeRecord(DataInputStream in, int version) throws IOException { + return deserializeEdit(in, new HashMap(), version); + } + + @Override + public Object getRecordIdentifier(final MapWaliRecord record) { + return record.getKey(); + } + + @Override + public UpdateType getUpdateType(final MapWaliRecord record) { + return record.getUpdateType(); + } + + @Override + public String getLocation(final MapWaliRecord record) { + return null; + } + + @Override + public int getVersion() { + return 1; + } + } +} \ No newline at end of file diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java new file mode 100644 index 0000000000..10139f149b --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/map/SimpleMapCache.java @@ -0,0 +1,165 @@ +/* + * 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.distributed.cache.server.map; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.SortedMap; +import java.util.concurrent.ConcurrentSkipListMap; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.nifi.distributed.cache.server.EvictionPolicy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SimpleMapCache implements MapCache { + private static final Logger logger = LoggerFactory.getLogger(SimpleMapCache.class); + + private final Map cache = new HashMap<>(); + private final SortedMap inverseCacheMap; + + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + private final Lock writeLock = rwLock.writeLock(); + + private final String serviceIdentifier; + + private final int maxSize; + + public SimpleMapCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) { + // need to change to ConcurrentMap as this is modified when only the readLock is held + inverseCacheMap = new ConcurrentSkipListMap<>(evictionPolicy.getComparator()); + this.serviceIdentifier = serviceIdentifier; + this.maxSize = maxSize; + } + + @Override + public String toString() { + return "SimpleSetCache[service id=" + serviceIdentifier + "]"; + } + + // don't need synchronized because this method is only called when the writeLock is held, and all + // public methods obtain either the read or write lock + private MapCacheRecord evict() { + if ( cache.size() < maxSize ) { + return null; + } + + final MapCacheRecord recordToEvict = inverseCacheMap.firstKey(); + final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict); + cache.remove(valueToEvict); + + if ( logger.isDebugEnabled() ) { + logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8)); + } + + return recordToEvict; + } + + @Override + public MapPutResult putIfAbsent(final ByteBuffer key, final ByteBuffer value) { + writeLock.lock(); + try { + final MapCacheRecord record = cache.get(key); + if ( record == null ) { + // Record is null. We will add. + final MapCacheRecord evicted = evict(); + final MapCacheRecord newRecord = new MapCacheRecord(key, value); + cache.put(key, newRecord); + inverseCacheMap.put(newRecord, key); + + if ( evicted == null ) { + return new MapPutResult(true, key, value, null, null, null); + } else { + return new MapPutResult(true, key, value, null, evicted.getKey(), evicted.getValue()); + } + } + + // Record is not null. Increment hit count and return result indicating that record was not added. + inverseCacheMap.remove(record); + record.hit(); + inverseCacheMap.put(record, key); + + return new MapPutResult(false, key, value, record.getValue(), null, null); + } finally { + writeLock.unlock(); + } + } + + @Override + public boolean containsKey(final ByteBuffer key) { + readLock.lock(); + try { + final MapCacheRecord record = cache.get(key); + if ( record == null ) { + return false; + } + + inverseCacheMap.remove(record); + record.hit(); + inverseCacheMap.put(record, key); + + return true; + } finally { + readLock.unlock(); + } + } + + @Override + public ByteBuffer get(final ByteBuffer key) { + readLock.lock(); + try { + final MapCacheRecord record = cache.get(key); + if ( record == null ) { + return null; + } + + inverseCacheMap.remove(record); + record.hit(); + inverseCacheMap.put(record, key); + + return record.getValue(); + } finally { + readLock.unlock(); + } + } + + @Override + public ByteBuffer remove(ByteBuffer key) throws IOException { + writeLock.lock(); + try { + final MapCacheRecord record = cache.remove(key); + if (record == null) { + return null; + } + inverseCacheMap.remove(record); + return record.getValue(); + } finally { + writeLock.unlock(); + } + } + + @Override + public void shutdown() throws IOException { + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java new file mode 100644 index 0000000000..4d75fc00d6 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/PersistentSetCache.java @@ -0,0 +1,194 @@ +/* + * 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.distributed.cache.server.set; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; + +import org.wali.MinimalLockingWriteAheadLog; +import org.wali.SerDe; +import org.wali.UpdateType; +import org.wali.WriteAheadRepository; + +public class PersistentSetCache implements SetCache { + + private final SetCache wrapped; + private final WriteAheadRepository wali; + + private final AtomicLong modifications = new AtomicLong(0L); + + public PersistentSetCache(final String serviceIdentifier, final File persistencePath, final SetCache cacheToWrap) throws IOException { + wali = new MinimalLockingWriteAheadLog<>(persistencePath.toPath(), 1, new Serde(), null); + wrapped = cacheToWrap; + } + + public synchronized void restore() throws IOException { + final Collection recovered = wali.recoverRecords(); + for ( final SetRecord record : recovered ) { + if ( record.getUpdateType() == UpdateType.CREATE ) { + addIfAbsent(record.getBuffer()); + } + } + } + + @Override + public synchronized SetCacheResult remove(final ByteBuffer value) throws IOException { + final SetCacheResult removeResult = wrapped.remove(value); + if ( removeResult.getResult() ) { + final SetRecord record = new SetRecord(UpdateType.DELETE, value); + final List records = new ArrayList<>(); + records.add(record); + wali.update(records, false); + + final long modCount = modifications.getAndIncrement(); + if ( modCount > 0 && modCount % 1000 == 0 ) { + wali.checkpoint(); + } + } + + return removeResult; + } + + @Override + public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) throws IOException { + final SetCacheResult addResult = wrapped.addIfAbsent(value); + if ( addResult.getResult() ) { + final SetRecord record = new SetRecord(UpdateType.CREATE, value); + final List records = new ArrayList<>(); + records.add(record); + + final SetCacheRecord evictedRecord = addResult.getEvictedRecord(); + if ( evictedRecord != null ) { + records.add(new SetRecord(UpdateType.DELETE, evictedRecord.getValue())); + } + + wali.update(records, false); + + final long modCount = modifications.getAndIncrement(); + if ( modCount > 0 && modCount % 1000 == 0 ) { + wali.checkpoint(); + } + } + + return addResult; + } + + @Override + public synchronized SetCacheResult contains(final ByteBuffer value) throws IOException { + return wrapped.contains(value); + } + + @Override + public void shutdown() throws IOException { + wali.shutdown(); + } + + private static class SetRecord { + private final UpdateType updateType; + private final ByteBuffer value; + + public SetRecord(final UpdateType updateType, final ByteBuffer value) { + this.updateType = updateType; + this.value = value; + } + + public UpdateType getUpdateType() { + return updateType; + } + + public ByteBuffer getBuffer() { + return value; + } + + public byte[] getData() { + return value.array(); + } + } + + private static class Serde implements SerDe { + + @Override + public void serializeEdit(final SetRecord previousRecordState, final SetRecord newRecordState, final DataOutputStream out) throws IOException { + final UpdateType updateType = newRecordState.getUpdateType(); + if ( updateType == UpdateType.DELETE ) { + out.write(0); + } else { + out.write(1); + } + + final byte[] data = newRecordState.getData(); + out.writeInt(data.length); + out.write(newRecordState.getData()); + } + + @Override + public void serializeRecord(SetRecord record, DataOutputStream out) throws IOException { + serializeEdit(null, record, out); + } + + @Override + public SetRecord deserializeEdit(final DataInputStream in, final Map currentRecordStates, final int version) throws IOException { + final int value = in.read(); + if ( value < 0 ) { + throw new EOFException(); + } + + final UpdateType updateType = (value == 0 ? UpdateType.DELETE : UpdateType.CREATE); + + final int size = in.readInt(); + final byte[] data = new byte[size]; + in.readFully(data); + + return new SetRecord(updateType, ByteBuffer.wrap(data)); + } + + @Override + public SetRecord deserializeRecord(DataInputStream in, int version) throws IOException { + return deserializeEdit(in, new HashMap(), version); + } + + @Override + public Object getRecordIdentifier(final SetRecord record) { + return record.getBuffer(); + } + + @Override + public UpdateType getUpdateType(final SetRecord record) { + return record.getUpdateType(); + } + + @Override + public String getLocation(final SetRecord record) { + return null; + } + + @Override + public int getVersion() { + return 1; + } + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java new file mode 100644 index 0000000000..bf6ae3e51c --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCache.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.distributed.cache.server.set; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public interface SetCache { + + SetCacheResult remove(ByteBuffer value) throws IOException; + SetCacheResult addIfAbsent(ByteBuffer value) throws IOException; + SetCacheResult contains(ByteBuffer value) throws IOException; + void shutdown() throws IOException; + +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java new file mode 100644 index 0000000000..20b6fae889 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheRecord.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.distributed.cache.server.set; + +import java.nio.ByteBuffer; + +import org.apache.nifi.distributed.cache.server.CacheRecord; + +public class SetCacheRecord extends CacheRecord { + private final ByteBuffer value; + + public SetCacheRecord(final ByteBuffer value) { + this.value = value; + } + + public ByteBuffer getValue() { + return value; + } + + @Override + public int hashCode() { + return value.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if ( this == obj ) { + return true; + } + + if (obj instanceof SetCacheRecord) { + return value.equals(((SetCacheRecord) obj).value); + } + return false; + } + + @Override + public String toString() { + return "SetCacheRecord[value=" + new String(value.array()) + ", hitCount=" + getHitCount() + "]"; + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java new file mode 100644 index 0000000000..732c4f003b --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SetCacheResult.java @@ -0,0 +1,43 @@ +/* + * 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.distributed.cache.server.set; + + + +public class SetCacheResult { + private final boolean result; + private final SetCacheRecord stats; + private final SetCacheRecord evictedRecord; + + public SetCacheResult(final boolean result, final SetCacheRecord stats, final SetCacheRecord evictedRecord) { + this.result = result; + this.stats = stats; + this.evictedRecord = evictedRecord; + } + + public boolean getResult() { + return result; + } + + public SetCacheRecord getRecord() { + return stats; + } + + public SetCacheRecord getEvictedRecord() { + return evictedRecord; + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java new file mode 100644 index 0000000000..77d648154a --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/java/org/apache/nifi/distributed/cache/server/set/SimpleSetCache.java @@ -0,0 +1,117 @@ +/* + * 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.distributed.cache.server.set; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.SortedMap; +import java.util.TreeMap; + +import org.apache.nifi.distributed.cache.server.EvictionPolicy; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class SimpleSetCache implements SetCache { + private static final Logger logger = LoggerFactory.getLogger(SimpleSetCache.class); + + private final Map cache = new HashMap<>(); + private final SortedMap inverseCacheMap; + + private final String serviceIdentifier; + + private final int maxSize; + + public SimpleSetCache(final String serviceIdentifier, final int maxSize, final EvictionPolicy evictionPolicy) { + inverseCacheMap = new TreeMap<>(evictionPolicy.getComparator()); + this.serviceIdentifier = serviceIdentifier; + this.maxSize = maxSize; + } + + private synchronized SetCacheRecord evict() { + if ( cache.size() < maxSize ) { + return null; + } + + final SetCacheRecord recordToEvict = inverseCacheMap.firstKey(); + final ByteBuffer valueToEvict = inverseCacheMap.remove(recordToEvict); + cache.remove(valueToEvict); + + if ( logger.isDebugEnabled() ) { + logger.debug("Evicting value {} from cache", new String(valueToEvict.array(), StandardCharsets.UTF_8)); + } + + return recordToEvict; + } + + @Override + public synchronized SetCacheResult addIfAbsent(final ByteBuffer value) { + final SetCacheRecord record = cache.get(value); + if ( record == null ) { + final SetCacheRecord evicted = evict(); + final SetCacheRecord newRecord = new SetCacheRecord(value); + cache.put(value, newRecord); + inverseCacheMap.put(newRecord, value); + return new SetCacheResult(true, newRecord, evicted); + } else { + // We have to remove the record and add it again in order to cause the Map to stay sorted + inverseCacheMap.remove(record); + record.hit(); + inverseCacheMap.put(record, value); + + return new SetCacheResult(false, record, null); + } + } + + @Override + public synchronized SetCacheResult contains(final ByteBuffer value) { + final SetCacheRecord record = cache.get(value); + if ( record == null ) { + return new SetCacheResult(false, null, null); + } else { + // We have to remove the record and add it again in order to cause the Map to stay sorted + inverseCacheMap.remove(record); + record.hit(); + inverseCacheMap.put(record, value); + + return new SetCacheResult(true, record, null); + } + } + + @Override + public synchronized SetCacheResult remove(final ByteBuffer value) { + final SetCacheRecord record = cache.remove(value); + if ( record == null ) { + return new SetCacheResult(false, null, null); + } else { + inverseCacheMap.remove(record); + return new SetCacheResult(true, record, null); + } + } + + @Override + public String toString() { + return "SimpleSetCache[service id=" + serviceIdentifier + "]"; + } + + @Override + public void shutdown() throws IOException { + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 0000000000..0509c7cca1 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.distributed.cache.server.DistributedSetCacheServer +org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer \ No newline at end of file diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html new file mode 100644 index 0000000000..dca3aa1f32 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/main/resources/docs/org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer/index.html @@ -0,0 +1,82 @@ + + + + + +Distributed Map Cache Client Service + + + + +

Description:

+ +

A Controller Service that starts an embedded server and listens for connections from clients. The + server provides the ability to query the cache, add data to the cache, and remove data from the cache.

+ + + +

+ Properties: +

+

In the list below, the names of required properties appear + in bold. Any other properties (not in bold) are considered optional. + If a property has a default value, it is indicated. If a property + supports the use of the NiFi Expression Language (or simply, + "expression language"), that is also indicated.

+ +
    +
  • Port +
      +
    • The port to listen on for incoming connections
    • +
    • Default value: 4557
    • +
    • Supports expression language: false
    • +
  • +
  • SSL Context Service +
      +
    • If specified, this service will be used to create an SSL Context that will be used to secure communications; if not specified, communications will not be secure
    • +
    • Default value: no default
    • +
    • Supports expression language: false
    • +
  • +
  • Maximum Cache Entries +
      +
    • The maximum number of cache entries that the cache can hold +
    • Default value: 10,000
    • +
    • Supports expression language: false
    • +
  • +
  • Eviction Strategy +
      +
    • Determines which strategy should be used to evict values from the cache to make room for new entries. Valid values: + Least Frequently Used, Least Recently Used, and First In, First Out +
    • Default value: Least Frequently Used
    • +
    • Supports expression language: false
    • +
  • +
  • Persistence Directory +
      +
    • If specified, the cache will be persisted in the given directory; if not specified, the cache will be in-memory only
    • +
    • Default value: no default (in-memory)
    • +
    • Supports expression language: true - JVM and System Properties Only
    • +
  • +
+ + + See Also: + + + + diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java new file mode 100644 index 0000000000..b5f3fd6f41 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/java/org/apache/nifi/distributed/cache/server/TestServerAndClient.java @@ -0,0 +1,530 @@ +/* + * 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.distributed.cache.server; + +import org.apache.nifi.distributed.cache.server.DistributedSetCacheServer; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.net.ConnectException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.distributed.cache.client.Deserializer; +import org.apache.nifi.distributed.cache.client.DistributedMapCacheClientService; +import org.apache.nifi.distributed.cache.client.DistributedSetCacheClientService; +import org.apache.nifi.distributed.cache.client.Serializer; +import org.apache.nifi.distributed.cache.client.exception.DeserializationException; +import org.apache.nifi.distributed.cache.server.map.DistributedMapCacheServer; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.ssl.SSLContextService.ClientAuth; +import org.apache.nifi.ssl.StandardSSLContextService; +import org.apache.nifi.util.MockConfigurationContext; +import org.apache.nifi.util.MockControllerServiceInitializationContext; + +import org.apache.commons.lang3.SerializationException; +import org.junit.Ignore; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class TestServerAndClient { + + private static Logger LOGGER; + + static { + System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info"); + System.setProperty("org.slf4j.simpleLogger.showDateTime", "true"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.AbstractCacheServer", "debug"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.client.DistributedMapCacheClientService", "debug"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.distributed.cache.server.TestServerAndClient", "debug"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.remote.io.socket.ssl.SSLSocketChannel", "trace"); + LOGGER = LoggerFactory.getLogger(TestServerAndClient.class); + } + + @Test + public void testNonPersistentSetServerAndClient() throws InitializationException, IOException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create server + final DistributedSetCacheServer server = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final Map serverProperties = new HashMap<>(); + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + final DistributedSetCacheClientService client = createClient(); + final Serializer serializer = new StringSerializer(); + final boolean added = client.addIfAbsent("test", serializer); + assertTrue(added); + + final boolean contains = client.contains("test", serializer); + assertTrue(contains); + + final boolean addedAgain = client.addIfAbsent("test", serializer); + assertFalse(addedAgain); + + final boolean removed = client.remove("test", serializer); + assertTrue(removed); + + final boolean containedAfterRemove = client.contains("test", serializer); + assertFalse(containedAfterRemove); + + server.shutdownServer(); + } + + @Test + public void testPersistentSetServerAndClient() throws InitializationException, IOException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create server + final DistributedSetCacheServer server = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final File dataFile = new File("target/cache-data"); + deleteRecursively(dataFile); + + final Map serverProperties = new HashMap<>(); + serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath()); + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + final DistributedSetCacheClientService client = createClient(); + final Serializer serializer = new StringSerializer(); + final boolean added = client.addIfAbsent("test", serializer); + final boolean added2 = client.addIfAbsent("test2", serializer); + assertTrue(added); + assertTrue(added2); + + final boolean contains = client.contains("test", serializer); + final boolean contains2 = client.contains("test2", serializer); + assertTrue(contains); + assertTrue(contains2); + + final boolean addedAgain = client.addIfAbsent("test", serializer); + assertFalse(addedAgain); + + final boolean removed = client.remove("test", serializer); + assertTrue(removed); + + final boolean containedAfterRemove = client.contains("test", serializer); + assertFalse(containedAfterRemove); + + server.shutdownServer(); + + final DistributedSetCacheServer newServer = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2"); + newServer.initialize(newServerInitContext); + + final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties, + newServerInitContext.getControllerServiceLookup()); + newServer.startServer(newServerContext); + + assertFalse(client.contains("test", serializer)); + assertTrue(client.contains("test2", serializer)); + + newServer.shutdownServer(); + } + + @Test + public void testPersistentSetServerAndClientWithLFUEvictions() throws InitializationException, IOException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create server + final DistributedSetCacheServer server = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final File dataFile = new File("target/cache-data"); + deleteRecursively(dataFile); + + final Map serverProperties = new HashMap<>(); + serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath()); + serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3"); + + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + final DistributedSetCacheClientService client = createClient(); + final Serializer serializer = new StringSerializer(); + final boolean added = client.addIfAbsent("test", serializer); + waitABit(); + final boolean added2 = client.addIfAbsent("test2", serializer); + waitABit(); + final boolean added3 = client.addIfAbsent("test3", serializer); + waitABit(); + assertTrue(added); + assertTrue(added2); + assertTrue(added3); + + final boolean contains = client.contains("test", serializer); + final boolean contains2 = client.contains("test2", serializer); + assertTrue(contains); + assertTrue(contains2); + + final boolean addedAgain = client.addIfAbsent("test", serializer); + assertFalse(addedAgain); + + final boolean added4 = client.addIfAbsent("test4", serializer); + assertTrue(added4); + + // ensure that added3 was evicted because it was used least frequently + assertFalse(client.contains("test3", serializer)); + + server.shutdownServer(); + + final DistributedSetCacheServer newServer = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2"); + newServer.initialize(newServerInitContext); + + final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties, + newServerInitContext.getControllerServiceLookup()); + newServer.startServer(newServerContext); + + assertTrue(client.contains("test", serializer)); + assertTrue(client.contains("test2", serializer)); + assertFalse(client.contains("test3", serializer)); + assertTrue(client.contains("test4", serializer)); + + newServer.shutdownServer(); + } + + @Test + public void testPersistentSetServerAndClientWithFIFOEvictions() throws InitializationException, IOException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create server + final DistributedSetCacheServer server = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final File dataFile = new File("target/cache-data"); + deleteRecursively(dataFile); + + final Map serverProperties = new HashMap<>(); + serverProperties.put(DistributedSetCacheServer.PERSISTENCE_PATH, dataFile.getAbsolutePath()); + serverProperties.put(DistributedSetCacheServer.MAX_CACHE_ENTRIES, "3"); + serverProperties.put(DistributedSetCacheServer.EVICTION_POLICY, DistributedSetCacheServer.EVICTION_STRATEGY_FIFO); + + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + final DistributedSetCacheClientService client = createClient(); + final Serializer serializer = new StringSerializer(); + + // add 3 entries to the cache. But, if we add too fast, we'll have the same millisecond + // for the entry time so we don't know which entry will be evicted. So we wait a few millis in between + final boolean added = client.addIfAbsent("test", serializer); + waitABit(); + final boolean added2 = client.addIfAbsent("test2", serializer); + waitABit(); + final boolean added3 = client.addIfAbsent("test3", serializer); + waitABit(); + + assertTrue(added); + assertTrue(added2); + assertTrue(added3); + + final boolean contains = client.contains("test", serializer); + final boolean contains2 = client.contains("test2", serializer); + assertTrue(contains); + assertTrue(contains2); + + final boolean addedAgain = client.addIfAbsent("test", serializer); + assertFalse(addedAgain); + + final boolean added4 = client.addIfAbsent("test4", serializer); + assertTrue(added4); + + // ensure that added3 was evicted because it was used least frequently + assertFalse(client.contains("test", serializer)); + assertTrue(client.contains("test3", serializer)); + + server.shutdownServer(); + + final DistributedSetCacheServer newServer = new DistributedSetCacheServer(); + MockControllerServiceInitializationContext newServerInitContext = new MockControllerServiceInitializationContext(newServer, "server2"); + newServer.initialize(newServerInitContext); + + final MockConfigurationContext newServerContext = new MockConfigurationContext(serverProperties, + newServerInitContext.getControllerServiceLookup()); + newServer.startServer(newServerContext); + + assertFalse(client.contains("test", serializer)); + assertTrue(client.contains("test2", serializer)); + assertTrue(client.contains("test3", serializer)); + assertTrue(client.contains("test4", serializer)); + + newServer.shutdownServer(); + } + + @Test + public void testNonPersistentMapServerAndClient() throws InitializationException, IOException, InterruptedException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create server + final DistributedMapCacheServer server = new DistributedMapCacheServer(); + MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final Map serverProperties = new HashMap<>(); + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + DistributedMapCacheClientService client = new DistributedMapCacheClientService(); + MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client"); + client.initialize(clientInitContext); + + final Map clientProperties = new HashMap<>(); + clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost"); + clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs"); + MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup()); + client.cacheConfig(clientContext); + final Serializer valueSerializer = new StringSerializer(); + final Serializer keySerializer = new StringSerializer(); + final Deserializer deserializer = new StringDeserializer(); + + final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer); + assertEquals(null, original); + LOGGER.debug("end getAndPutIfAbsent"); + + final boolean contains = client.containsKey("testKey", keySerializer); + assertTrue(contains); + LOGGER.debug("end containsKey"); + + final boolean added = client.putIfAbsent("testKey", "test", keySerializer, valueSerializer); + assertFalse(added); + LOGGER.debug("end putIfAbsent"); + + final String originalAfterPut = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer); + assertEquals("test", originalAfterPut); + LOGGER.debug("end getAndPutIfAbsent"); + + final boolean removed = client.remove("testKey", keySerializer); + assertTrue(removed); + LOGGER.debug("end remove"); + + final boolean containedAfterRemove = client.containsKey("testKey", keySerializer); + assertFalse(containedAfterRemove); + + client.putIfAbsent("testKey", "test", keySerializer, valueSerializer); + client.close(); + try { + client.containsKey("testKey", keySerializer); + fail("Should be closed and not accessible"); + } catch (Exception e) { + + } + client = null; + clientInitContext = null; + clientContext = null; + + DistributedMapCacheClientService client2 = new DistributedMapCacheClientService(); + + MockControllerServiceInitializationContext clientInitContext2 = new MockControllerServiceInitializationContext(client2, "client2"); + client2.initialize(clientInitContext2); + + MockConfigurationContext clientContext2 = new MockConfigurationContext(clientProperties, + clientInitContext2.getControllerServiceLookup()); + client2.cacheConfig(clientContext2); + assertFalse(client2.putIfAbsent("testKey", "test", keySerializer, valueSerializer)); + assertTrue(client2.containsKey("testKey", keySerializer)); + server.shutdownServer(); + Thread.sleep(1000); + try { + client2.containsKey("testKey", keySerializer); + fail("Should have blown exception!"); + } catch (ConnectException e) { + client2 = null; + clientContext2 = null; + clientInitContext2 = null; + } + Thread.sleep(2000); + System.gc(); + LOGGER.debug("end testNonPersistentMapServerAndClient"); + } + + @Test + public void testClientTermination() throws InitializationException, IOException, InterruptedException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create server + final DistributedMapCacheServer server = new DistributedMapCacheServer(); + MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final Map serverProperties = new HashMap<>(); + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + DistributedMapCacheClientService client = new DistributedMapCacheClientService(); + MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client"); + client.initialize(clientInitContext); + + final Map clientProperties = new HashMap<>(); + clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost"); + clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs"); + MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup()); + client.cacheConfig(clientContext); + final Serializer valueSerializer = new StringSerializer(); + final Serializer keySerializer = new StringSerializer(); + final Deserializer deserializer = new StringDeserializer(); + + final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer); + assertEquals(null, original); + + final boolean contains = client.containsKey("testKey", keySerializer); + assertTrue(contains); + + final boolean added = client.putIfAbsent("testKey", "test", keySerializer, valueSerializer); + assertFalse(added); + + final String originalAfterPut = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer); + assertEquals("test", originalAfterPut); + + final boolean removed = client.remove("testKey", keySerializer); + assertTrue(removed); + + final boolean containedAfterRemove = client.containsKey("testKey", keySerializer); + assertFalse(containedAfterRemove); + + client = null; + clientInitContext = null; + clientContext = null; + Thread.sleep(2000); + System.gc(); + server.shutdownServer(); + } + + @Ignore + @Test + public void testSSLWith2RequestsWithServerTimeout() throws InitializationException, IOException, InterruptedException { + LOGGER.info("Testing " + Thread.currentThread().getStackTrace()[1].getMethodName()); + // Create SSLContext Service + final StandardSSLContextService sslService = new StandardSSLContextService(); + final MockControllerServiceInitializationContext sslServerInitContext = new MockControllerServiceInitializationContext(sslService, + "ssl-context"); + sslService.initialize(sslServerInitContext); + + final Map sslServerProps = new HashMap<>(); + sslServerProps.put(StandardSSLContextService.KEYSTORE, "src/test/resources/localhost-ks.jks"); + sslServerProps.put(StandardSSLContextService.KEYSTORE_PASSWORD, "localtest"); + sslServerProps.put(StandardSSLContextService.KEYSTORE_TYPE, "JKS"); + sslServerProps.put(StandardSSLContextService.TRUSTSTORE, "src/test/resources/localhost-ts.jks"); + sslServerProps.put(StandardSSLContextService.TRUSTSTORE_PASSWORD, "localtest"); + sslServerProps.put(StandardSSLContextService.TRUSTSTORE_TYPE, "JKS"); + MockConfigurationContext sslServerContext = new MockConfigurationContext(sslServerProps, sslServerInitContext); + sslService.onConfigured(sslServerContext); + sslService.createSSLContext(ClientAuth.REQUIRED); + // Create server + final DistributedMapCacheServer server = new DistributedMapCacheServer(); + final MockControllerServiceInitializationContext serverInitContext = new MockControllerServiceInitializationContext(server, "server"); + server.initialize(serverInitContext); + + final Map serverProperties = new HashMap<>(); + serverProperties.put(DistributedMapCacheServer.SSL_CONTEXT_SERVICE, "ssl-context"); + final MockConfigurationContext serverContext = new MockConfigurationContext(serverProperties, serverInitContext.getControllerServiceLookup()); + server.startServer(serverContext); + + DistributedMapCacheClientService client = new DistributedMapCacheClientService(); + MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client"); + client.initialize(clientInitContext); + + final Map clientProperties = new HashMap<>(); + clientProperties.put(DistributedMapCacheClientService.HOSTNAME, "localhost"); + clientProperties.put(DistributedMapCacheClientService.COMMUNICATIONS_TIMEOUT, "360 secs"); + clientProperties.put(DistributedMapCacheClientService.SSL_CONTEXT_SERVICE, "ssl-context"); + MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup()); + client.cacheConfig(clientContext); + final Serializer valueSerializer = new StringSerializer(); + final Serializer keySerializer = new StringSerializer(); + final Deserializer deserializer = new StringDeserializer(); + + final String original = client.getAndPutIfAbsent("testKey", "test", keySerializer, valueSerializer, deserializer); + assertEquals(null, original); + + Thread.sleep(30000); + try { + final boolean contains = client.containsKey("testKey", keySerializer); + assertTrue(contains); + } catch (IOException e) { + // this is due to the server timing out in the middle of this request + assertTrue(e.getMessage().contains("Channel is closed")); + } + + server.shutdownServer(); + } + + private void waitABit() { + try { + Thread.sleep(10L); + } catch (final InterruptedException e) { + } + } + + private DistributedSetCacheClientService createClient() throws InitializationException { + final DistributedSetCacheClientService client = new DistributedSetCacheClientService(); + MockControllerServiceInitializationContext clientInitContext = new MockControllerServiceInitializationContext(client, "client"); + client.initialize(clientInitContext); + + final Map clientProperties = new HashMap<>(); + clientProperties.put(DistributedSetCacheClientService.HOSTNAME, "localhost"); + final MockConfigurationContext clientContext = new MockConfigurationContext(clientProperties, clientInitContext.getControllerServiceLookup()); + client.onConfigured(clientContext); + + return client; + } + + private static class StringSerializer implements Serializer { + @Override + public void serialize(final String value, final OutputStream output) throws SerializationException, IOException { + output.write(value.getBytes(StandardCharsets.UTF_8)); + } + } + + private static class StringDeserializer implements Deserializer { + @Override + public String deserialize(final byte[] input) throws DeserializationException, IOException { + return (input.length == 0) ? null : new String(input, StandardCharsets.UTF_8); + } + } + + private static void deleteRecursively(final File dataFile) throws IOException { + if (dataFile == null || !dataFile.exists()) { + return; + } + + final File[] children = dataFile.listFiles(); + for (final File child : children) { + if (child.isDirectory()) { + deleteRecursively(child); + } else { + for (int i = 0; i < 100 && child.exists(); i++) { + child.delete(); + } + + if (child.exists()) { + throw new IOException("Could not delete " + dataFile.getAbsolutePath()); + } + } + } + } +} diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks new file mode 100755 index 0000000000..81be31d710 Binary files /dev/null and b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ks.jks differ diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks new file mode 100755 index 0000000000..820e1e19fe Binary files /dev/null and b/nar-bundles/distributed-cache-services-bundle/distributed-cache-server/src/test/resources/localhost-ts.jks differ diff --git a/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml b/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml new file mode 100644 index 0000000000..75cab34441 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/distributed-cache-services-nar/pom.xml @@ -0,0 +1,49 @@ + + + 4.0.0 + + org.apache.nifi + distributed-cache-services-bundle + 0.0.1-SNAPSHOT + + + distributed-cache-services-nar + Distributed Cache Services NAR + nar + + + + org.apache.nifi + standard-services-api-nar + nar + + + org.apache.nifi + distributed-cache-client-service + ${project.version} + + + org.apache.nifi + distributed-cache-protocol + ${project.version} + + + org.apache.nifi + distributed-cache-server + ${project.version} + + + diff --git a/nar-bundles/distributed-cache-services-bundle/pom.xml b/nar-bundles/distributed-cache-services-bundle/pom.xml new file mode 100644 index 0000000000..dcfa541e22 --- /dev/null +++ b/nar-bundles/distributed-cache-services-bundle/pom.xml @@ -0,0 +1,83 @@ + + + 4.0.0 + + + org.apache.nifi + standard-services-api-bundle + 0.0.1-SNAPSHOT + + + distributed-cache-services-bundle + 0.0.1-SNAPSHOT + Distributed Cache Services Bundle + pom + + + distributed-cache-protocol + distributed-cache-client-service + distributed-cache-server + distributed-cache-services-nar + + + + + + + org.apache.nifi + distributed-cache-client-service-api + ${standard.services.api.version} + + + org.apache.nifi + ssl-context-service-api + ${standard.services.api.version} + + + org.apache.nifi + nifi-processor-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-stream-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + remote-communications-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-mock + 0.0.1-SNAPSHOT + test + + + wali + wali + 3.0.0-SNAPSHOT + + + org.apache.nifi + ssl-context-service + 0.0.1-SNAPSHOT + test + + + + \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml b/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml new file mode 100644 index 0000000000..a5d3d11bb3 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/pom.xml @@ -0,0 +1,81 @@ + + + 4.0.0 + + org.apache.nifi + execute-script-bundle + 0.0.1-SNAPSHOT + + execute-script-processors + NiFi Processors to Run Scripts + NiFi Script Execution Processors + + + + org.jruby + jruby + + + jnr-netdb + com.github.jnr + + + jnr-posix + com.github.jnr + + + jffi + com.github.jnr + + + nailgun-server + com.martiansoftware + + + + + + org.python + jython-standalone + + + org.apache.nifi + nifi-mock + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + commons-io + commons-io + + + org.apache.nifi + nifi-core-flowfile-attributes + + + org.apache.nifi + nifi-stream-utils + + + + + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java new file mode 100644 index 0000000000..9058cf41fc --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/processors/script/ExecuteScript.java @@ -0,0 +1,566 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.script; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.script.ScriptException; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.io.BufferedOutputStream; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessorInitializationContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.annotation.CapabilityDescription; +import org.apache.nifi.processor.annotation.EventDriven; +import org.apache.nifi.processor.annotation.Tags; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.io.StreamCallback; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.scripting.ConverterScript; +import org.apache.nifi.scripting.ReaderScript; +import org.apache.nifi.scripting.Script; +import org.apache.nifi.scripting.ScriptFactory; +import org.apache.nifi.scripting.WriterScript; + +/** + * + *

Description:

+ *

+ * This processor provides the capability to execute scripts in various + * scripting languages, and passes into the scripts the input stream and output + * stream(s) representing an incoming flow file and any created flow files. The + * processor is designed to be thread safe, so multiple concurrent tasks may + * execute against a single script. The processor provides a framework which + * enables script writers to implement 3 different types of scripts: + *

    + * ReaderScript - which enables stream-based reading of a FlowFile's + * content
    WriterScript - which enables stream-based reading and + * writing/modifying of a FlowFile's content
    ConverterScript - which + * enables stream-based reading a FlowFile's content and stream-based writing to + * newly created FlowFiles
    + *
+ * Presently, the processor supports 3 scripting languages: Ruby, Python, and + * JavaScript. The processor is built on the javax.script API which enables + * ScriptEngine discovery, thread management, and encapsulates much of the low + * level bridging-code that enables Java to Script language integration. Thus, + * it is designed to be easily extended to other scripting languages.
The + * attributes of a FlowFile and properties of the Processor are exposed to the + * script by either a variable in the base class or a getter method. A script + * may declare new Processor Properties and different Relationships via + * overriding the getPropertyDescriptors and getRelationships methods, + * respectively. + *

+ *

+ * Properties: + *

+ *

+ * In the list below, the names of required properties appear in bold. Any other + * properties (not in bold) are considered optional. If a property has a default + * value, it is indicated. If a property supports the use of the NiFi Expression + * Language (or simply, "expression language"), that is also indicated. Of + * particular note: This processor allows scripts to define additional Processor + * properties, which will not be initially visible. Once the processor's + * configuration is validated, script defined properties will become visible, + * and may affect the validity of the processor. + *

+ *
    + *
  • + * Script File Name + *
      + *
    • Script location, can be relative or absolute path.
    • + *
    • Default value: no default
    • + *
    • Supports expression language: false
    • + *
    + *
  • + *
  • + * Script Check Interval + *
      + *
    • The time period between checking for updates to a script.
    • + *
    • Default value: 15 sec
    • + *
    • Supports expression language: false
    • + *
    + *
  • + *
+ * + *

+ * Relationships: + *

+ *

+ * The initial 'out of the box' relationships are below. Of particular note is + * the ability of a script to change the set of relationships. However, any + * relationships defined by the script will not be visible until the processor's + * configuration has been validated. Once done, new relationships will become + * visible. + *

+ *
    + *
  • + * success + *
      + *
    • Used when a file is successfully processed by a script.
    • + *
    + *
  • + *
  • + * failure + *
      + *
    • Used when an error occurs while processing a file with a script.
    • + *
    + *
  • + *
+ * + *

+ * Example Scripts: + *

+ *
    + * JavaScript example - the 'with' statement imports packages defined in the + * framework. Since the 'instance' variable is intended to be local scope (not + * global), it must be named 'instance' as it it not passed back to the + * processor upon script evaluation and must be fetched. If you make it global, + * you can name it whatever you'd like...but this is intended to be + * multi-threaded so do so at your own risk. Presently, there are issues with + * the JavaScript scripting engine that prevent sub-classing the base classes in + * the Processor's Java framework. So, what is actually happening is an instance + * of the ReaderScript is created with a provided callback object. When we are + * able to move to a more competent scripting engine, the code below will remain + * the same, but the 'instance' variable will actually be a sub-class of + * ReaderScript. + * + *
    + *               with (Scripting) {
    + *                 var instance = new ReaderScript({
    + *                     route : function(input) {
    + *                         var str = IOUtils.toString(input);
    + *                         var expr = instance.getProperty("expr");
    + *                         filename = instance.attributes.get("filename");
    + *                         instance.setAttribute("filename", filename + ".modified");
    + *                         if (str.match(expr)) {
    + *                             return Script.FAIL_RELATIONSHIP;
    + *                         } else {
    + *                             return Script.SUCCESS_RELATIONSHIP;
    + *                         }
    + *                     }
    + *                 });
    + *               }
    + * 
    + * + * Ruby example - the 'OutputStreamHandler' is an interface which is called when + * creating flow files. + * + *
    + *                 java_import 'org.apache.nifi.scripting.OutputStreamHandler'
    + *                 class SimpleConverter < ConverterScript
    + *                   field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
    + *
    + *                   def convert(input)
    + *                     in_io = input.to_io
    + *                     createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
    + *                         out_io = out.to_io
    + *                         out_io << in_io.readline.to_java_bytes
    + *                         out_io.close
    + *                         logger.debug("Wrote data to failure...this message logged with logger from super class")
    + *                       end)
    + *
    + *                     createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
    + *                         out_io = out.to_io
    + *                         in_io.each_line { |line|
    + *                           out_io << line
    + *                         }
    + *                         out_io.close
    + *                         logger.debug("Wrote data to success...this message logged with logger from super class")
    + *                       end)
    + *                     in_io.close
    + *                   end
    + *
    + *                 end
    + *
    + *                 $logger.debug("Creating SimpleConverter...this message logged with logger from shared variables")
    + *                 SimpleConverter.new
    + * 
    + * + * Python example - The difficulty with Python is that it does not return + * objects upon script evaluation, so the instance of the Script class must be + * fetched by name. Thus, you must define a variable called 'instance'. + * + *
    + *                 import re
    + *
    + *                 class RoutingReader(ReaderScript):
    + *                     A = Relationship.Builder().name("a").description("some good stuff").build()
    + *                     B = Relationship.Builder().name("b").description("some other stuff").build()
    + *                     C = Relationship.Builder().name("c").description("some bad stuff").build()
    + *
    + *                     def getRelationships(self):
    + *                         return [self.A,self.B,self.C]
    + *
    + *                     def getExceptionRoute(self):
    + *                         return self.C
    + *
    + *                     def route( self, input ):
    + *                         for line in FileUtil.wrap(input):
    + *                             if re.match("^bad", line, re.IGNORECASE):
    + *                                 return self.B
    + *                             if re.match("^sed", line):
    + *                                 raise RuntimeError("That's no good!")
    + *
    + *                         return self.A
    + *
    + *                 instance = RoutingReader()
    + * 
    + * + *
+ *

+ * Shared Variables + *

+ *
    + *
  • logger : global scope
  • + *
  • properties : local/instance scope
  • + *
+ *

+ * Script API: + *

+ *
    + *
  • getAttribute(String) : String
  • + *
  • getAttributes() : Map(String,String)
  • + *
  • getExceptionRoute() : Relationship
  • + *
  • getFileName() : String
  • + *
  • getFlowFileEntryDate() : Calendar
  • + *
  • getFlowFileSize() : long
  • + *
  • getProperties() : Map(String, String)
  • + *
  • getProperty(String) : String
  • + *
  • getPropertyDescriptors() : List(PropertyDescriptor)
  • + *
  • getRelationships() : Collection(Relationship)
  • + *
  • getRoute() : Relationship
  • + *
  • setRoute(Relationship)
  • + *
  • setAttribute(String, String)
  • + *
  • validate() : Collection(String)
  • + *
+ *

+ * ReaderScript API: + *

+ *
    + *
  • route(InputStream) : Relationship
  • + *
+ *

+ * WriterScript API: + *

+ *
    + *
  • process(InputStream, OutputStream)
  • + *
+ *

+ * ConverterScript API: + *

+ *
    + *
  • convert(InputStream)
  • + *
  • createFlowFile(String, Relationship, OutputStreamHandler)
  • + *
+ *

+ * OutputStreamHandler API: + *

+ *
    + *
  • write(OutputStream)
  • + *
+ */ +@EventDriven +@Tags({"script", "ruby", "python", "javascript", "execute"}) +@CapabilityDescription("Execute scripts in various scripting languages, and passes into the scripts the input stream and output stream(s) " + + "representing an incoming flow file and any created flow files.") +public class ExecuteScript extends AbstractProcessor { + + private final AtomicBoolean doCustomValidate = new AtomicBoolean(true); + private final AtomicReference> relationships = new AtomicReference<>(); + private final AtomicReference> propertyDescriptors = new AtomicReference<>(); + private volatile ScriptFactory scriptFactory; + private volatile Relationship exceptionRoute; + + /** + * Script location, can be relative or absolute path -- passed as-is to + * {@link File#File(String) File constructor} + */ + public static final PropertyDescriptor SCRIPT_FILE_NAME = new PropertyDescriptor.Builder() + .name("Script File Name") + .description("Script location, can be relative or absolute path") + .required(true) + .addValidator(new Validator() { + + @Override + public ValidationResult validate(String subject, String input, ValidationContext context) { + ValidationResult result = StandardValidators.FILE_EXISTS_VALIDATOR.validate(subject, input, context); + if (result.isValid()) { + int dotPos = input.lastIndexOf('.'); + if (dotPos < 1) { + result = new ValidationResult.Builder() + .subject(subject) + .valid(false) + .explanation("Filename must have an extension") + .input(input) + .build(); + } + } + return result; + } + }) + .build(); + + static final PropertyDescriptor SCRIPT_CHECK_INTERVAL = new PropertyDescriptor.Builder() + .name("Script Check Interval") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .description("The time period between checking for updates to a script") + .required(true) + .defaultValue("15 sec") + .build(); + + @Override + protected void init(ProcessorInitializationContext context) { + Set empty = Collections.emptySet(); + relationships.set(empty); + ArrayList propDescs = new ArrayList<>(); + propDescs.add(SCRIPT_FILE_NAME); + propDescs.add(SCRIPT_CHECK_INTERVAL); + propertyDescriptors.set(Collections.unmodifiableList(propDescs)); + scriptFactory = new ScriptFactory(getLogger()); + } + + @Override + public List getSupportedPropertyDescriptors() { + return propertyDescriptors.get(); + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .dynamic(true) + .addValidator(Validator.VALID) + .build(); + } + + @Override + public void onPropertyModified(PropertyDescriptor descriptor, String oldValue, String newValue) { + doCustomValidate.set(true); + } + + @Override + public Set getRelationships() { + return relationships.get(); + } + + /** + * Called by framework. + * + * Returns a list of reasons why this processor cannot be run. + * @return + */ + @Override + protected Collection customValidate(ValidationContext validationContext) { + if (doCustomValidate.getAndSet(false)) { + long interval = validationContext.getProperty(SCRIPT_CHECK_INTERVAL).asTimePeriod(TimeUnit.MILLISECONDS); + scriptFactory.setScriptCheckIntervalMS(interval); + List results = new ArrayList<>(); + String file = validationContext.getProperty(SCRIPT_FILE_NAME).getValue(); + try { + Script s = scriptFactory.getScript(file); + + // set the relationships of the processor + relationships.set(new HashSet<>(s.getRelationships())); + + // need to get script's prop. descs. and validate. May, or may not, have dynamic + // props already...depends if this is the first time the processor is being configured. + Map properties = validationContext.getProperties(); + + // need to compare props, if any, against script-expected props that are required. + // script may be expecting required props that are not known, or some props may have invalid + // values. + // processor may be configured with dynamic props that the script will use...but does not declare which would + // be a bad thing + List scriptPropDescs = s.getPropertyDescriptors(); + getLogger().debug("Script is {}", new Object[]{s}); + getLogger().debug("Script file name is {}", new Object[]{s.getFileName()}); + getLogger().debug("Script Prop Descs are: {}", new Object[]{scriptPropDescs.toString()}); + getLogger().debug("Thread is: {}", new Object[]{Thread.currentThread().toString()}); + for (PropertyDescriptor propDesc : scriptPropDescs) { + // need to check for missing props + if (propDesc.isRequired() && !properties.containsKey(propDesc)) { + results.add(new ValidationResult.Builder() + .subject("Script Properties") + .valid(false) + .explanation("Missing Property " + propDesc.getName()) + .build()); + + // need to validate current value against script provided validator + } else if (properties.containsKey(propDesc)) { + String value = properties.get(propDesc); + ValidationResult result = propDesc.validate(value, validationContext); + if (!result.isValid()) { + results.add(result); + } + } // else it is an optional prop according to the script and it is not specified by + // the configuration of the processor + } + + // need to update the known prop desc's with what we just got from the script + List pds = new ArrayList<>(propertyDescriptors.get()); + pds.addAll(scriptPropDescs); + propertyDescriptors.set(Collections.unmodifiableList(pds)); + + if (results.isEmpty()) { + // so needed props are supplied and individually validated, now validate script + Collection reasons; + reasons = s.validate(); + if (null == reasons) { + getLogger().warn("Script had invalid return value for validate(), ignoring."); + } else { + for (String reason : reasons) { + ValidationResult result = new ValidationResult.Builder() + .subject("ScriptValidation") + .valid(false) + .explanation(reason) + .build(); + results.add(result); + } + } + } + + // get the exception route + exceptionRoute = s.getExceptionRoute(); + + return results; + } catch (ScriptException | IOException | NoSuchMethodException e) { + doCustomValidate.set(true); + results.add(new ValidationResult.Builder() + .subject("ScriptValidation") + .valid(false) + .explanation("Cannot create script due to " + e.getMessage()) + .input(file) + .build()); + getLogger().error("Cannot create script due to " + e, e); + return results; + } + } + return null; + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; // fail-fast if there is no work to do + } + + final String scriptFileName = context.getProperty(SCRIPT_FILE_NAME).getValue(); + // doing this cloning because getProperties does not initialize props that have only their default values + // must do a getProperty for that value to be initialized + Map props = new HashMap<>(); + for (PropertyDescriptor propDesc : context.getProperties().keySet()) { + if (propDesc.isExpressionLanguageSupported()) { + props.put(propDesc.getName(), context.getProperty(propDesc).evaluateAttributeExpressions(flowFile).getValue()); + } else { + props.put(propDesc.getName(), context.getProperty(propDesc).getValue()); + } + } + Script script = null; + try { + final Script finalScript = scriptFactory.getScript(scriptFileName, props, flowFile); + script = finalScript; + if (finalScript instanceof ReaderScript) { + session.read(flowFile, new InputStreamCallback() { + + @Override + public void process(InputStream in) throws IOException { + try { + ((ReaderScript) finalScript).process(new BufferedInputStream(in)); + } catch (NoSuchMethodException | ScriptException e) { + getLogger().error("Failed to execute ReaderScript", e); + throw new IOException(e); + } + } + }); + } else if (finalScript instanceof WriterScript) { + flowFile = session.write(flowFile, new StreamCallback() { + + @Override + public void process(InputStream in, OutputStream out) throws IOException { + try { + ((WriterScript) finalScript).process(new BufferedInputStream(in), new BufferedOutputStream(out)); + out.flush(); + } catch (NoSuchMethodException | ScriptException e) { + getLogger().error("Failed to execute WriterScript", e); + throw new IOException(e); + } + } + }); + } else if (finalScript instanceof ConverterScript) { + ((ConverterScript) finalScript).process(session); + + // Note that these scripts don't pass the incoming FF through, + // they always create new outputs + session.remove(flowFile); + return; + } else { + // only thing we can do is assume script has already run and done it's thing, so just transfer the incoming + // flowfile + getLogger().debug("Successfully executed script from {}", new Object[]{scriptFileName}); + } + + // update flow file attributes + flowFile = session.putAllAttributes(flowFile, finalScript.getAttributes()); + Relationship route = finalScript.getRoute(); + if (null == route) { + session.remove(flowFile); + getLogger().info("Removing flowfile {}", new Object[]{flowFile}); + } else { + session.transfer(flowFile, route); + getLogger().info("Transferring flowfile {} to {}", new Object[]{flowFile, route}); + } + } catch (ScriptException | IOException e) { + getLogger().error("Failed to create script from {} with flowFile {}. Rolling back session.", + new Object[]{scriptFileName, flowFile}, e); + throw new ProcessException(e); + } catch (Exception e) { + if (null != script) { + getLogger().error("Failed to execute script from {}. Transferring flow file {} to {}", + new Object[]{scriptFileName, flowFile, exceptionRoute}, e); + session.transfer(flowFile, exceptionRoute); + } else { + getLogger().error("Failed to execute script from {} with flowFile {}. Rolling back session", + new Object[]{scriptFileName, flowFile}, e); + throw new ProcessException(e); + } + } + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java new file mode 100644 index 0000000000..7be47a8411 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ConverterScript.java @@ -0,0 +1,131 @@ +/* + * 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.scripting; + +import java.io.ByteArrayOutputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; + +import javax.script.Invocable; +import javax.script.ScriptException; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.io.OutputStreamCallback; + +/** + *

+ * Script authors should extend this class if they want to perform complex + * conversions in a NiFi processor. + *

+ * + *

+ * Scripts must implement {@link #convert(FileInputStream)}. This method may + * create new FlowFiles and pass them to one or more routes. The input FlowFile + * will be removed from the repository after execution of this method completes. + *

+ * + *

+ * In general, the {@link #convert(FileInputStream)} will read from the supplied + * stream, then create one or more output sinks and route the result to the + * relationship of choice using + * {@link #routeStream(ByteArrayOutputStream, String, String)} or + * {@link #routeBytes(byte[], String, String)}. + * + *

+ * Implement {@link #getProcessorRelationships()} to allow writing to + * relationships other than success and failure. The + * {@link #getRoute()} superclass method is *not* used by Converter Scripts. + *

+ * + */ +public class ConverterScript extends Script { + + private ProcessSession session; // used to create files + private Object convertCallback; + + public ConverterScript() { + + } + + public ConverterScript(Object... callbacks) { + super(callbacks); + for (Object callback : callbacks) { + if (callback instanceof Map) { + convertCallback = convertCallback == null && ((Map) callback).containsKey("convert") ? callback : convertCallback; + } + } + } + + // Subclasses should implement this to define basic logic + protected void convert(InputStream stream) throws NoSuchMethodException, ScriptException { + if (convertCallback != null) { + ((Invocable) engine).invokeMethod(convertCallback, "convert", stream); + } + } + + /** + * Owning processor uses this method to kick off handling of a single file + * + * @param aSession the owning processor's Repository (needed to make new + * files) + */ + public void process(ProcessSession aSession) { + this.session = aSession; + this.session.read(this.flowFile, new InputStreamCallback() { + + @Override + public void process(InputStream in) throws IOException { + BufferedInputStream stream = new BufferedInputStream(in); + try { + convert(stream); + } catch (NoSuchMethodException | ScriptException e) { + logger.error("Failed to execute 'convert' function in script", e); + throw new IOException(e); + } + } + }); + } + + // this should go back to protected once we get Nashorn + public void createFlowFile(final String flowFileName, final Relationship relationship, final OutputStreamHandler handler) { + FlowFile result = session.create(this.flowFile); + result = session.putAttribute(result, CoreAttributes.FILENAME.key(), flowFileName); + try { + result = session.write(result, new OutputStreamCallback() { + + @Override + public void process(OutputStream out) throws IOException { + handler.write(out); + } + }); + this.logger.info("Transfer flow file {} to {}", new Object[]{result, relationship}); + session.transfer(result, relationship); + } catch (Exception e) { + this.logger.error("Could not create new flow file from script", e); + session.remove(result); + } + } + +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.java new file mode 100644 index 0000000000..883b688fac --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JRubyScriptFactory.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.scripting; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.io.FileUtils; + +public enum JRubyScriptFactory { + + INSTANCE; + + private static final String PRELOADS = "include Java\n" + + "\n" + + "java_import 'org.apache.nifi.components.PropertyDescriptor'\n" + + "java_import 'org.apache.nifi.components.Validator'\n" + + "java_import 'org.apache.nifi.processor.util.StandardValidators'\n" + + "java_import 'org.apache.nifi.processor.Relationship'\n" + + "java_import 'org.apache.nifi.logging.ProcessorLog'\n" + + "java_import 'org.apache.nifi.scripting.ReaderScript'\n" + + "java_import 'org.apache.nifi.scripting.WriterScript'\n" + + "java_import 'org.apache.nifi.scripting.ConverterScript'\n" + + "\n"; + + public String getScript(File scriptFile) throws IOException { + StringBuilder sb = new StringBuilder(); + sb.append(PRELOADS) + .append(FileUtils.readFileToString(scriptFile, "UTF-8")); + return sb.toString(); + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java new file mode 100644 index 0000000000..774fb1f35c --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JavaScriptScriptFactory.java @@ -0,0 +1,56 @@ +/* + * 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.scripting; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; + +public enum JavaScriptScriptFactory { + + INSTANCE; + + private static final String PRELOADS = "var Scripting = JavaImporter(\n" + + " Packages.org.apache.nifi.components,\n" + + " Packages.org.apache.nifi.processor.util,\n" + + " Packages.org.apache.nifi.processor,\n" + + " Packages.org.apache.nifi.logging,\n" + + " Packages.org.apache.nifi.scripting,\n" + + " Packages.org.apache.commons.io);\n" + + "var readFile = function (file) {\n" + + " var script = Packages.org.apache.commons.io.FileUtils.readFileToString(" + + " new java.io.File($PATH, file)" + + " );\n" + + " return \"\" + script;\n" + + "}\n" + + "var require = function (file){\n" + + " var exports={}, module={};\n" + + " module.__defineGetter__('id', function(){return file;});" + + " eval(readFile(file));\n" + + " return exports;\n" + + "}\n"; + + public String getScript(File scriptFile) throws IOException { + StringBuilder sb = new StringBuilder(); + final String parent = StringUtils.replace(scriptFile.getParent(), "\\", "/"); + sb.append(PRELOADS).append("var $PATH = \"").append(parent).append("\"\n") + .append(FileUtils.readFileToString(scriptFile, "UTF-8")); + return sb.toString(); + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java new file mode 100644 index 0000000000..6b40b5e50c --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/JythonScriptFactory.java @@ -0,0 +1,45 @@ +/* + * 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.scripting; + +import java.io.File; +import java.io.IOException; + +import org.apache.commons.io.FileUtils; + +public enum JythonScriptFactory { + + INSTANCE; + + private final static String PRELOADS = "from org.python.core.util import FileUtil\n" + + "from org.apache.nifi.components import PropertyDescriptor\n" + + "from org.apache.nifi.components import Validator\n" + + "from org.apache.nifi.processor.util import StandardValidators\n" + + "from org.apache.nifi.processor import Relationship\n" + + "from org.apache.nifi.logging import ProcessorLog\n" + + "from org.apache.nifi.scripting import ReaderScript\n" + + "from org.apache.nifi.scripting import WriterScript\n" + + "from org.apache.nifi.scripting import ConverterScript\n"; + + public String getScript(File scriptFile) throws IOException { + StringBuilder sb = new StringBuilder(); + sb.append(PRELOADS) + .append(FileUtils.readFileToString(scriptFile, "UTF-8")); + + return sb.toString(); + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java new file mode 100644 index 0000000000..d87972279a --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/OutputStreamHandler.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.scripting; + +import java.io.OutputStream; + +public interface OutputStreamHandler { + + void write(OutputStream out); +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java new file mode 100644 index 0000000000..b1d89c0b59 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ReaderScript.java @@ -0,0 +1,79 @@ +/* + * 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.scripting; + +import java.io.InputStream; +import java.util.Map; + +import javax.script.Invocable; +import javax.script.ScriptException; + +import org.apache.nifi.processor.Relationship; + +/** + *

+ * Script authors should extend this class if they want to follow the "reader" + * paradigm for NiFi processors. + *

+ * + *

+ * User scripts should implement {@link #route(InputStream)}. route + * uses a returned relationship name to determine where FlowFiles go. Scripts + * may also implement {@link #getProcessorRelationships()} to specify available + * relationship names. + *

+ * + */ +public class ReaderScript extends Script { + + private Object routeCallback; + + public ReaderScript(Object... callbacks) { + super(callbacks); + for (Object callback : callbacks) { + if (callback instanceof Map) { + routeCallback = routeCallback == null && ((Map) callback).containsKey("route") ? callback : routeCallback; + } + } + } + + public ReaderScript() { + + } + + // Simple helper + public void process(InputStream input) throws NoSuchMethodException, ScriptException { + lastRoute = route(input); + } + + /** + * Subclasses should examine the provided inputstream, then determine which + * relationship the file will be sent down and return its name. + * + * + * @param in a Java InputStream containing the incoming FlowFile. + * @return a relationship name + * @throws ScriptException + * @throws NoSuchMethodException + */ + public Relationship route(InputStream in) throws NoSuchMethodException, ScriptException { + Relationship relationship = null; + Invocable invocable = (Invocable) this.engine; + relationship = (Relationship) invocable.invokeMethod(routeCallback, "route", in); + return relationship; + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java new file mode 100644 index 0000000000..786f54114b --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/Script.java @@ -0,0 +1,303 @@ +/* + * 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.scripting; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import javax.script.Invocable; +import javax.script.ScriptEngine; +import javax.script.ScriptException; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ProcessorLog; +import org.apache.nifi.processor.Relationship; + +/** + *

+ * Base class for all scripts. In this framework, only ScriptEngines that + * implement javax.script.Invocable are supported. + * + *

+ * + */ +public class Script { + + public static final Relationship SUCCESS_RELATIONSHIP = new Relationship.Builder() + .name("success") + .description("Destination of successfully created flow files") + .build(); + public static final Relationship FAIL_RELATIONSHIP = new Relationship.Builder() + .name("failure") + .description("Destination of flow files when a error occurs in the script") + .build(); + + static final Set RELATIONSHIPS; + + static { + Set rels = new HashSet<>(); + rels.add(FAIL_RELATIONSHIP); + rels.add(SUCCESS_RELATIONSHIP); + RELATIONSHIPS = Collections.unmodifiableSet(rels); + } + + FlowFile flowFile = null; + ScriptEngine engine = null; + + protected Map properties = new HashMap<>(); + protected Relationship lastRoute = SUCCESS_RELATIONSHIP; + protected ProcessorLog logger; + protected String scriptFileName; + protected Map attributes = new HashMap<>(); + protected long flowFileSize = 0; + protected long flowFileEntryDate = System.currentTimeMillis(); + + // the following are needed due to an inadequate JavaScript ScriptEngine. It will not allow + // subclassing a Java Class, only implementing a Java Interface. So, the syntax of JavaScript + // scripts looks like subclassing, but actually is just constructing a Script instance and + // passing in functions as args to the constructor. When we move to Nashorn JavaScript ScriptEngine + // in Java 8, we can get rid of these and revert the subclasses of this class to abstract. + protected Object propDescCallback; + protected Object relationshipsCallback; + protected Object validateCallback; + protected Object exceptionRouteCallback; + + /** + * Create a Script without any parameters + */ + public Script() { + } + + public Script(Object... callbacks) { + for (Object callback : callbacks) { + if (callback instanceof Map) { + propDescCallback = propDescCallback == null && ((Map) callback).containsKey("getPropertyDescriptors") ? callback + : propDescCallback; + relationshipsCallback = relationshipsCallback == null && ((Map) callback).containsKey("getRelationships") ? callback + : relationshipsCallback; + validateCallback = validateCallback == null && ((Map) callback).containsKey("validate") ? callback : validateCallback; + exceptionRouteCallback = exceptionRouteCallback == null && ((Map) callback).containsKey("getExceptionRoute") ? callback + : exceptionRouteCallback; + } + } + } + + /** + * Specify a set of properties with corresponding NiFi validators. + * + * Subclasses that do not override this method will still have access to all + * properties via the "properties" field + * + * @return a list of PropertyDescriptors + * @throws ScriptException + * @throws NoSuchMethodException + */ + @SuppressWarnings("unchecked") + public List getPropertyDescriptors() throws NoSuchMethodException, ScriptException { + if (propDescCallback != null) { + return (List) ((Invocable) engine).invokeMethod(propDescCallback, "getPropertyDescriptors", (Object) null); + } + return Collections.emptyList(); + } + + /** + * Specify a set of reasons why this processor should be invalid. + * + * Subclasses that do not override this method will depend only on + * individual property validators as specified in + * {@link #getPropertyDescriptors()}. + * + * @return a Collection of messages to display to the user, or an empty + * Collection if the processor configuration is OK. + * @throws ScriptException + * @throws NoSuchMethodException + */ + @SuppressWarnings("unchecked") + public Collection validate() throws NoSuchMethodException, ScriptException { + if (validateCallback != null) { + return (Collection) ((Invocable) engine).invokeMethod(validateCallback, "validate", (Object) null); + } + return Collections.emptyList(); + } + + void setFlowFile(FlowFile ff) { + flowFile = ff; + if (null != ff) { + // have to clone because ff.getAttributes is unmodifiable + this.attributes = new HashMap<>(ff.getAttributes()); + this.flowFileSize = ff.getSize(); + this.flowFileEntryDate = ff.getEntryDate(); + } + } + + void setProperties(Map map) { + properties = new HashMap<>(map); + } + + /** + * Required to access entire properties map -- Jython (at least) won't let + * you read the member variable without a getter + * + * @return entire parameter map + */ + // change back to protected when we get nashorn + public Map getProperties() { + return properties; + } + + /** + * Get the named parameter. Some scripting languages make a method call + * easier than accessing a member field, so this is a convenience method to + * look up values in the properties field. + * + * @param key a hash key + * @return the value pointed at by the key specified + */ + public String getProperty(String key) { + return properties.get(key); + } + + /** + * Name the various relationships by which a file can leave this processor. + * Subclasses may override this method to change available relationships. + * + * @return a collection of relationship names + * @throws ScriptException + * @throws NoSuchMethodException + */ + @SuppressWarnings("unchecked") + public Collection getRelationships() throws NoSuchMethodException, ScriptException { + if (relationshipsCallback != null) { + return (Collection) ((Invocable) engine).invokeMethod(relationshipsCallback, "getRelationships", (Object) null); + } + return RELATIONSHIPS; + } + + /** + * Determine what do with a file that has just been processed. + * + * After a script runs its "read" or "write" method, it should update the + * "lastRoute" field to specify the relationship to which the resulting file + * will be sent. + * + * @return a relationship name + */ + public Relationship getRoute() { + return lastRoute; + } + + // Required because of a potential issue in Rhino -- protected methods are visible in + // subclasses but protected fields (like "lastRoute") are not + // change back to protected when we get nashorn + public void setRoute(Relationship route) { + lastRoute = route; + } + + /** + * Determine where to send a file if an exception is thrown during + * processing. + * + * Subclasses may override this method to use a different relationship, or + * to determine the relationship dynamically. Returning null causes the file + * to be deleted instead. + * + * Defaults to "failure". + * + * @return the name of the relationship to use in event of an exception, or + * null to delete the file. + * @throws ScriptException + * @throws NoSuchMethodException + */ + public Relationship getExceptionRoute() throws NoSuchMethodException, ScriptException { + if (exceptionRouteCallback != null) { + return (Relationship) ((Invocable) engine).invokeMethod(exceptionRouteCallback, "getExceptionRoute", (Object) null); + } + return FAIL_RELATIONSHIP; + } + + /* + * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to get + * the incoming flow file size. + */ + // Change back to protected when we get nashorn + public long getFlowFileSize() { + return flowFileSize; + } + + /* + * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to get + * entry date of the flow file. + */ + // Change back to protected when we get nashorn + public long getFlowFileEntryDate() { + return flowFileEntryDate; + } + + void setLogger(ProcessorLog logger) { + this.logger = logger; + } + + /* + * Required so that scripts in some languages can read access the attribute. Jython (at least) won't let you read the member + * variable without a getter + */ + protected ProcessorLog getLogger() { + return this.logger; + } + + void setFileName(String scriptFileName) { + this.scriptFileName = scriptFileName; + } + + public String getFileName() { + return this.scriptFileName; + } + + // this one's public because it's needed by ExecuteScript to update the flow file's attributes AFTER processing is done + public Map getAttributes() { + return this.attributes; + } + + /* + * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to look + * up values in the attributes field. + */ + // Change back to protected when we get nashorn + public String getAttribute(String key) { + return this.attributes.get(key); + } + + /* + * Some scripting languages make a method call easier than accessing a member field, so this is a convenience method to set + * key/value pairs in the attributes field. + */ + // Change back to protected when we get nashorn + public void setAttribute(String key, String value) { + this.attributes.put(key, value); + } + + void setEngine(ScriptEngine scriptEngine) { + this.engine = scriptEngine; + } + +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java new file mode 100644 index 0000000000..6f388869e0 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptEngineFactory.java @@ -0,0 +1,117 @@ +/* + * 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.scripting; + +import java.io.File; +import java.util.concurrent.ConcurrentHashMap; + +import javax.script.ScriptEngine; +import javax.script.ScriptEngineManager; +import javax.script.ScriptException; + +import org.apache.commons.lang3.StringUtils; +import org.jruby.embed.PropertyName; + +public class ScriptEngineFactory { + + private static final String THREADING = "THREADING"; + private static final String MULTITHREADED = "MULTITHREADED"; + private static final String STATELESS = "STATELESS"; + private static final String THREAD_ISOLATED = "THREAD-ISOLATED"; + final static ScriptEngineManager scriptEngMgr; + + static { + System.setProperty(PropertyName.LOCALCONTEXT_SCOPE.toString(), "singlethread"); + System.setProperty(PropertyName.COMPILEMODE.toString(), "jit"); + System.setProperty(PropertyName.COMPATVERSION.toString(), "JRuby1.9"); + System.setProperty(PropertyName.LOCALVARIABLE_BEHAVIOR.toString(), "transient"); + System.setProperty("compile.invokedynamic", "false"); + System.setProperty(PropertyName.LAZINESS.toString(), "true"); + scriptEngMgr = new ScriptEngineManager(); + } + final ConcurrentHashMap threadSafeEngines = new ConcurrentHashMap<>(); + + ScriptEngine getEngine(String extension) { + ScriptEngine engine = threadSafeEngines.get(extension); + if (null == engine) { + engine = scriptEngMgr.getEngineByExtension(extension); + if (null == engine) { + throw new IllegalArgumentException("No ScriptEngine exists for extension " + extension); + } + + Object threading = engine.getFactory().getParameter(THREADING); + // the MULTITHREADED status means that the scripts need to be careful about sharing state + if (THREAD_ISOLATED.equals(threading) || STATELESS.equals(threading) || MULTITHREADED.equals(threading)) { + ScriptEngine cachedEngine = threadSafeEngines.putIfAbsent(extension, engine); + if (null != cachedEngine) { + engine = cachedEngine; + } + } + } + return engine; + } + + ScriptEngine getNewEngine(File scriptFile, String extension) throws ScriptException { + ScriptEngine engine = scriptEngMgr.getEngineByExtension(extension); + if (null == engine) { + throw new IllegalArgumentException("No ScriptEngine exists for extension " + extension); + } + // Initialize some paths + StringBuilder sb = new StringBuilder(); + switch (extension) { + case "rb": + String parent = scriptFile.getParent(); + parent = StringUtils.replace(parent, "\\", "/"); + sb.append("$:.unshift '") + .append(parent) + .append("'\n") + .append("$:.unshift File.join '") + .append(parent) + .append("', 'lib'\n"); + engine.eval(sb.toString()); + + break; + case "py": + parent = scriptFile.getParent(); + parent = StringUtils.replace(parent, "\\", "/"); + String lib = parent + "/lib"; + sb.append("import sys\n").append("sys.path.append('").append(parent) + .append("')\n").append("sys.path.append('") + .append(lib) + .append("')\n") + .append("__file__ = '") + .append(scriptFile.getAbsolutePath()) + .append("'\n"); + engine.eval(sb.toString()); + break; + default: + break; + } + + Object threading = engine.getFactory().getParameter(THREADING); + // the MULTITHREADED status means that the scripts need to be careful about sharing state + if (THREAD_ISOLATED.equals(threading) || STATELESS.equals(threading) || MULTITHREADED.equals(threading)) { + // replace prior instance if any + threadSafeEngines.put(extension, engine); + } + return engine; + } + + boolean isThreadSafe(String scriptExtension) { + return threadSafeEngines.containsKey(scriptExtension); + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java new file mode 100644 index 0000000000..da18606505 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/ScriptFactory.java @@ -0,0 +1,269 @@ +/* + * 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.scripting; + +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.security.DigestInputStream; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock; +import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock; + +import javax.script.Bindings; +import javax.script.Compilable; +import javax.script.CompiledScript; +import javax.script.ScriptContext; +import javax.script.ScriptEngine; +import javax.script.ScriptException; +import javax.script.SimpleBindings; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.logging.ProcessorLog; + +import org.apache.commons.io.FileUtils; + +/** + * While this is a 'factory', it is not a singleton because we want a factory + * per processor. This factory has state, all of which belong to only one + * processor. + * + */ +public class ScriptFactory { + + private final ScriptEngineFactory engineFactory = new ScriptEngineFactory(); + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + private final ReadLock readLock = lock.readLock(); + private final WriteLock writeLock = lock.writeLock(); + private final ProcessorLog logger; + + private volatile CompiledScript compiledScript; + private volatile String scriptText; + private volatile byte[] md5Hash; + private volatile long lastTimeChecked; + private volatile String scriptFileName; + private volatile long scriptCheckIntervalMS = 15000; + + public ScriptFactory(ProcessorLog logger) { + this.logger = logger; + } + + public void setScriptCheckIntervalMS(long msecs) { + this.scriptCheckIntervalMS = msecs; + } + + /** + * @param aScriptFileName + * @param properties + * @param flowFile + * @return + * @throws IOException + * @throws ScriptException + */ + public Script getScript(final String aScriptFileName, final Map properties, final FlowFile flowFile) + throws IOException, ScriptException { + final Script instance; + long now = System.currentTimeMillis(); + readLock.lock(); + try { + if (!aScriptFileName.equals(this.scriptFileName)) { + readLock.unlock(); + writeLock.lock(); + try { + if (!aScriptFileName.equals(this.scriptFileName)) { + // need to get brand new engine + compiledScript = null; + this.md5Hash = getMD5Hash(aScriptFileName); + this.lastTimeChecked = now; + this.scriptFileName = aScriptFileName; + updateEngine(); + } // else another thread beat me to the change...so just get a script + } finally { + readLock.lock(); + writeLock.unlock(); + } + } else if (lastTimeChecked + scriptCheckIntervalMS < now) { + readLock.unlock(); + writeLock.lock(); + try { + if (lastTimeChecked + scriptCheckIntervalMS < now) { + byte[] md5 = getMD5Hash(this.scriptFileName); + if (!MessageDigest.isEqual(md5Hash, md5)) { + // need to get brand new engine + compiledScript = null; + updateEngine(); + this.md5Hash = md5; + } // else no change to script, so just update time checked + this.lastTimeChecked = now; + } // else another thread beat me to the check...so just get a script + } finally { + readLock.lock(); + writeLock.unlock(); + } + } + try { + instance = getScriptInstance(properties); + instance.setFileName(this.scriptFileName); + instance.setProperties(properties); + instance.setLogger(logger); + instance.setFlowFile(flowFile); + } catch (ScriptException e) { + // need to reset state to enable re-initialization + this.lastTimeChecked = 0; + this.scriptFileName = null; + throw e; + } + } finally { + readLock.unlock(); + } + + return instance; + + } + + public Script getScript(String aScriptFileName) throws ScriptException, IOException { + Map props = new HashMap<>(); + return getScript(aScriptFileName, props, null); + } + + private byte[] getMD5Hash(String aScriptFileName) throws FileNotFoundException, IOException { + byte[] messageDigest = null; + try (FileInputStream fis = new FileInputStream(aScriptFileName); + DigestInputStream dis = new DigestInputStream(new BufferedInputStream(fis), MessageDigest.getInstance("MD5"))) { + + byte[] bytes = new byte[8192]; + while (dis.read(bytes) != -1) { + // do nothing...just computing the md5 hash + } + messageDigest = dis.getMessageDigest().digest(); + } catch (NoSuchAlgorithmException swallow) { + // MD5 is a legitimate format + } + return messageDigest; + } + + private String getScriptText(File scriptFile, String extension) throws IOException { + final String script; + switch (extension) { + case "rb": + script = JRubyScriptFactory.INSTANCE.getScript(scriptFile); + break; + + case "js": + script = JavaScriptScriptFactory.INSTANCE.getScript(scriptFile); + break; + + case "py": + script = JythonScriptFactory.INSTANCE.getScript(scriptFile); + break; + + default: + script = FileUtils.readFileToString(scriptFile); + } + return script; + } + + private Script getScriptInstance(final Map properties) throws ScriptException { + + Map localThreadVariables = new HashMap<>(); + final String extension = getExtension(scriptFileName); + String loggerVariableKey = getVariableName("GLOBAL", "logger", extension); + localThreadVariables.put(loggerVariableKey, logger); + String propertiesVariableKey = getVariableName("INSTANCE", "properties", extension); + localThreadVariables.put(propertiesVariableKey, properties); + localThreadVariables.put(ScriptEngine.FILENAME, scriptFileName); + final Bindings bindings = new SimpleBindings(localThreadVariables); + final ScriptEngine scriptEngine = engineFactory.getEngine(extension); + Script instance; + if (compiledScript == null) { + instance = (Script) scriptEngine.eval(scriptText, bindings); + if (instance == null) { // which it will be for python and also for local variables in javascript + instance = (Script) scriptEngine.eval("instance", bindings); + } + } else { + instance = (Script) compiledScript.eval(bindings); + if (instance == null) { // which it will be for python and also for local variables in javascript + instance = (Script) compiledScript.getEngine().eval("instance", bindings); + } + } + instance.setEngine(scriptEngine); + return instance; + } + + /* + * Must have writeLock when calling this!!!! + */ + private void updateEngine() throws IOException, ScriptException { + final String extension = getExtension(scriptFileName); + // if engine is thread safe, it's being reused...if it's a JrubyEngine it + File scriptFile = new File(this.scriptFileName); + ScriptEngine scriptEngine = engineFactory.getNewEngine(scriptFile, extension); + scriptText = getScriptText(scriptFile, extension); + Map localThreadVariables = new HashMap<>(); + String loggerVariableKey = getVariableName("GLOBAL", "logger", extension); + localThreadVariables.put(loggerVariableKey, logger); + String propertiesVariableKey = getVariableName("INSTANCE", "properties", extension); + localThreadVariables.put(propertiesVariableKey, new HashMap()); + localThreadVariables.put(ScriptEngine.FILENAME, scriptFileName); + if (scriptEngine instanceof Compilable) { + Bindings bindings = new SimpleBindings(localThreadVariables); + scriptEngine.setBindings(bindings, ScriptContext.ENGINE_SCOPE); + compiledScript = ((Compilable) scriptEngine).compile(scriptText); + } + logger.debug("Updating Engine!!"); + } + + private String getVariableName(String scope, String variableName, String extension) { + String result; + switch (extension) { + case "rb": + switch (scope) { + case "GLOBAL": + result = '$' + variableName; + break; + case "INSTANCE": + result = '@' + variableName; + break; + default: + result = variableName; + break; + } + + break; + + default: + result = variableName; + break; + } + return result; + } + + private String getExtension(String aScriptFileName) { + int dotPos = aScriptFileName.lastIndexOf('.'); + if (dotPos < 1) { + throw new IllegalArgumentException("Script file name must have an extension"); + } + final String extension = aScriptFileName.substring(dotPos + 1); + return extension; + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java new file mode 100644 index 0000000000..7eef98b20d --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/java/org/apache/nifi/scripting/WriterScript.java @@ -0,0 +1,67 @@ +/* + * 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.scripting; + +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Map; + +import javax.script.Invocable; +import javax.script.ScriptException; + +/** + *

+ * Script authors should extend this class if they want to follow the + * "processCallback" paradigm for NiFi processors. + *

+ * + *

+ * At a minimum, scripts must implement + * process(FileInputStream, FileOutputStream). + *

+ * + *

+ * By default, all files processed will be sent to the relationship + * success, unless the scriptFileName raises an exception, in which + * case the file will be sent to failure. Implement + * {@link #getProcessorRelationships()} and/or {@link #getRoute()} to change + * this behavior. + *

+ * + */ +public class WriterScript extends Script { + + private Object processCallback; + + public WriterScript() { + + } + + public WriterScript(Object... callbacks) { + super(callbacks); + for (Object callback : callbacks) { + if (callback instanceof Map) { + processCallback = processCallback == null && ((Map) callback).containsKey("process") ? callback : processCallback; + } + } + } + + public void process(InputStream in, OutputStream out) throws NoSuchMethodException, ScriptException { + Invocable inv = (Invocable) engine; + inv.invokeMethod(processCallback, "process", in, out); + } +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 0000000000..20a3982520 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -0,0 +1,15 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.processors.script.ExecuteScript diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html new file mode 100644 index 0000000000..acb47c5ed5 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/main/resources/docs/org.apache.nifi.processors.script.ExecuteScript/index.html @@ -0,0 +1,264 @@ + + + + + + ExecuteScript + + + + + + +

Description:

+

+ This processor provides the capability to execute scripts in various scripting languages, and passes into the scripts + the input stream and output stream(s) representing an incoming flow file and any created flow files. The processor is designed to be + thread safe, so multiple concurrent tasks may execute against a single script. The processor provides a framework which enables + script writers to implement 3 different types of scripts: +

    + ReaderScript - which enables stream-based reading of a FlowFile's content
    + WriterScript - which enables stream-based reading and writing/modifying of a FlowFile's content
    + ConverterScript - which enables stream-based reading a FlowFile's content and stream-based writing to newly created FlowFiles
    +
+ Presently, the processor supports 3 scripting languages: Ruby, Python, and JavaScript. The processor is built on the + javax.script API which enables ScriptEngine discovery, thread management, and encapsulates much of the low level bridging-code that + enables Java to Script language integration. Thus, it is designed to be easily extended to other scripting languages.
+ The attributes of a FlowFile and properties of the Processor are exposed to the script by either a variable in the base class or + a getter method. A script may declare new Processor Properties and different Relationships via overriding the getPropertyDescriptors + and getRelationships methods, respectively. +

+ The processor provides some boilerplate script to aid in the creation of the three different types of scripts. For example, + the processor provides import statements for classes commonly used within a processor. +
+                'org.apache.nifi.components.PropertyDescriptor'
+                'org.apache.nifi.components.Validator'
+                'org.apache.nifi.processor.util.StandardValidators'
+                'org.apache.nifi.processor.Relationship'
+                'org.apache.nifi.logging.ProcessorLog'
+                'org.apache.nifi.scripting.ReaderScript'
+                'org.apache.nifi.scripting.WriterScript'
+                'org.apache.nifi.scripting.ConverterScript'
+    
+ The processor appends to the script's execution path the parent directory of the specified script file and a sub-directory + called 'lib', which may be useful for supporting scripts.

+

+ Shared Variables +

+The following variables are provided as shared variables for the scripts: +
    +
  • logger +
      +
    • The processor's logger
    • +
    • Scope is GLOBAL, thus in Ruby the syntax is $logger
    • +
    +
  • +
  • properties +
      +
    • A Map of the processor's configuration properties; key and value are strings
    • +
    • Scope is INSTANCE, thus in Ruby the syntax is @properties
    • +
    +
  • +
+

+ Properties: +

+

+ In the list below, the names of required properties appear in bold. Any other properties (not in bold) are considered + optional. If a property has a default value, it is indicated. If a property supports the use of the NiFi Expression Language + (or simply, "expression language"), that is also indicated. Of particular note: This processor allows scripts to define additional + Processor properties, which will not be initially visible. Once the processor's configuration is validated, script defined properties + will become visible, and may affect the validity of the processor. +

+
    +
  • + Script File Name +
      +
    • Script location, can be relative or absolute path.
    • +
    • Default value: no default
    • +
    • Supports expression language: false
    • +
    +
  • +
  • + Script Check Interval +
      +
    • The time period between checking for updates to a script.
    • +
    • Default value: 15 sec
    • +
    • Supports expression language: false
    • +
    +
  • +
+ +

+ Relationships: +

+

+ The initial 'out of the box' relationships are below. Of particular note is the ability of a script to change the set of + relationships. However, any relationships defined by the script will not be visible until the processor's configuration has been + validated. Once done, new relationships will become visible. +

+
    +
  • + success +
      +
    • Used when a file is successfully processed by a script.
    • +
    +
  • +
  • + failure +
      +
    • Used when an error occurs while processing a file with a script.
    • +
    +
  • +
+ +

+ Example Scripts: +

+
    + JavaScript example - the 'with' statement imports packages defined in the framework and limits the importing to the local scope, + rather than global. The 'Scripting' variable uses the JavaImporter class within JavaScript. Since the 'instance' variable is intended to + be local scope (not global), it must be named 'instance' as it it not passed back to the processor upon script evaluation and must be + fetched. If you make it global, you can name it whatever you'd like...but this is intended to be multi-threaded so do so at your own + risk.

    +Presently, there are issues with the JavaScript scripting engine that prevent sub-classing the base classes in the Processor's Java +framework. So, what is actually happening is an instance of the ReaderScript is created with a provided callback object. When we are able +to move to a more competent scripting engine (supposedly in Java 8), the code below will remain the same, but the 'instance' variable +will actually be a sub-class of ReaderScript. +
     
    +              with (Scripting) {
    +                var instance = new ReaderScript({
    +                    route : function(input) {
    +                        var str = IOUtils.toString(input);
    +                        var expr = instance.getProperty("expr");
    +                        filename = instance.attributes.get("filename");
    +                        instance.setAttribute("filename", filename + ".modified");
    +                        if (str.match(expr)) {
    +                            return Script.FAIL_RELATIONSHIP;
    +                        } else {
    +                            return Script.SUCCESS_RELATIONSHIP;
    +                        }
    +                    }
    +                });
    +              }
    +
    +Ruby example - the 'OutputStreamHandler' is an interface which is called when creating flow files. +
    +                java_import 'org.apache.nifi.scripting.OutputStreamHandler'
    +                class SimpleConverter < ConverterScript
    +                  field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes
    +                  
    +                  def convert(input)
    +                    in_io = input.to_io
    +                    createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
    +                        out_io = out.to_io
    +                        out_io << in_io.readline.to_java_bytes
    +                        out_io.close
    +                        logger.debug("Wrote data to failure...this message logged with logger from super class")
    +                      end)
    +                      
    +                    createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out|
    +                        out_io = out.to_io
    +                        in_io.each_line { |line|
    +                          out_io << line
    +                        }
    +                        out_io.close
    +                        logger.debug("Wrote data to success...this message logged with logger from super class")
    +                      end)
    +                    in_io.close
    +                  end
    +                   
    +                end
    +                
    +                $logger.debug("Creating SimpleConverter...this message logged with logger from shared variables")
    +                SimpleConverter.new
    +
    +Python example - The difficulty with Python is that it does not return objects upon script evaluation, so the instance of the Script +class must be fetched by name. Thus, you must define a variable called 'instance'. +
    +                import re
    +                
    +                class RoutingReader(ReaderScript):
    +                    A = Relationship.Builder().name("a").description("some good stuff").build()
    +                    B = Relationship.Builder().name("b").description("some other stuff").build()
    +                    C = Relationship.Builder().name("c").description("some bad stuff").build()
    +                    
    +                    def getRelationships(self):
    +                        return [self.A,self.B,self.C]
    +                  
    +                    def getExceptionRoute(self):
    +                        return self.C
    +                  
    +                    def route( self, input ):
    +                        logger.info("Executing route")
    +                        for line in FileUtil.wrap(input):
    +                            if re.match("^bad", line, re.IGNORECASE):
    +                                return self.B
    +                            if re.match("^sed", line):
    +                                raise RuntimeError("That's no good!")
    +                
    +                        return self.A
    +                logger.debug("Constructing instance")
    +                instance = RoutingReader()
    +            
    +
    +
+

+ Script API: +

+
    +
  • getAttribute(String) : String
  • +
  • getAttributes() : Map(String,String)
  • +
  • getExceptionRoute() : Relationship
  • +
  • getFileName() : String
  • +
  • getFlowFileEntryDate() : Calendar
  • +
  • getFlowFileSize() : long
  • +
  • getProperties() : Map(String, String)
  • +
  • getProperty(String) : String
  • +
  • getPropertyDescriptors() : List(PropertyDescriptor)
  • +
  • getRelationships() : Collection(Relationship)
  • +
  • getRoute() : Relationship
  • +
  • setRoute(Relationship)
  • +
  • setAttribute(String, String)
  • +
  • validate() : Collection(String)
  • +
+

+ ReaderScript API: +

+
    +
  • route(InputStream) : Relationship
  • +
+

+ WriterScript API: +

+
    +
  • process(InputStream, OutputStream)
  • +
+

+ ConverterScript API: +

+
    +
  • convert(InputStream)
  • +
  • createFlowFile(String, Relationship, OutputStreamHandler)
  • +
+

+ OutputStreamHandler API: +

+
    +
  • write(OutputStream)
  • +
+ + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java new file mode 100644 index 0000000000..3a34769316 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/java/org/apache/nifi/processors/script/TestExecuteScript.java @@ -0,0 +1,939 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.script; + +import org.apache.nifi.processors.script.ExecuteScript; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; + +import org.apache.commons.io.FileUtils; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * @author unattributed + * + */ +public class TestExecuteScript { + + static Logger LOG; + + static { + System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info"); + System.setProperty("org.slf4j.simpleLogger.showDateTime", "true"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.script.ExecuteScript", "trace"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.script.TestExecuteScript", "debug"); + System.setProperty("org.slf4j.simpleLogger.log.nifi.processors.AbstractProcessor", "debug"); + LOG = LoggerFactory.getLogger(TestExecuteScript.class); + } + + private TestRunner controller; + + private final String multiline = "Lorem ipsum dolor sit amet,\n" + + "consectetur adipisicing elit,\n" + + "sed do eiusmod tempor incididunt ut labore et dolore magna aliqua.\n" + + "Ut enim ad minim veniam,\n" + + "quis nostrud exercitation ullamco laboris nisi ut aliquip ex ea commodo consequat.\n" + + "Duis aute irure dolor in reprehenderit in voluptate velit esse cillum dolore eu fugiat nulla pariatur.\n" + + "Excepteur sint occaecat cupidatat non proident,\n" + + "sunt in culpa qui officia deserunt mollit anim id est laborum."; + + /** + * Create a mock SingleProcessorController using our processor and pass data + * to it via byte array. Returns the Sink that provides access to any files + * that pass out of the processor + */ + @Before + public void setupEach() throws IOException { + controller = TestRunners.newTestRunner(ExecuteScript.class); + controller.setValidateExpressionUsage(false); + + // copy all scripts to target directory and run from there. some python + // scripts create .class files that end up in src/test/resources. + FileUtils.copyDirectory(new File("src/test/resources"), new File("target/test-scripts")); + } + + // Fail if the specified relationship does not contain exactly one file + // with the expected value + private void assertRelationshipContents(String expected, String relationship) { + controller.assertTransferCount(relationship, 1); + MockFlowFile ff = controller.getFlowFilesForRelationship(relationship).get(0); + ff.assertContentEquals(expected); + } + + // Fail if the specified relationship does not contain specified number of files + // with the expected value + private void assertRelationshipContents(String expected, String relationship, int count) { + controller.assertTransferCount(relationship, count); + MockFlowFile ff = controller.getFlowFilesForRelationship(relationship).get(count - 1); + ff.assertContentEquals(expected); + } + + // //////////////////////////////////// + // General tests + @Test(expected = IllegalArgumentException.class) + public void failOnBadName() { + LOG.info("Supplying bad script file names"); + + // None of these should result in actually setting the property, because they're non-existent / bad files + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "not/really.rb"); + controller.assertNotValid(); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "fakey/fake.js"); + controller.assertNotValid(); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "pom.xml"); + controller.assertNotValid(); + } + + // //////////////////////////////////// + // Ruby script tests + @Test + public void testSimpleReadR() { + LOG.info("Ruby script: fail file based on reading contents"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.rb"); + controller.setThreadCount(2); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + + controller.getFlowFilesForRelationship("success").get(0).assertAttributeEquals("filename", "NewFileNameFromReadTest"); + } + + @Test + public void testParamReadR() { + LOG.info("Ruby script: Failing file based on reading contents"); + + Map attrs1 = new HashMap<>(); + attrs1.put("filename", "StuffIsFine.txt"); + Map attrs2 = new HashMap<>(); + attrs2.put("filename", "multiline.txt"); + controller.enqueue("This stuff is fine".getBytes(), attrs1); + controller.enqueue(multiline.getBytes(), attrs2); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb"); + controller.setProperty("expr", "rehenderit"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testWriteLastLineR() { + LOG.info("Running Ruby script to output last line of file"); + + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.rb"); + controller.run(); + + List files = controller.getFlowFilesForRelationship("success"); + + assertEquals("Process did not generate an output file", 1, files.size()); + + byte[] blob = files.get(0).toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File had more than one line", 1, lines.length); + assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]); + } + + @Test + public void testWriteOptionalParametersR() { + LOG.info("Ruby script that uses optional parameters"); + + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.rb"); + controller.setProperty("repeat", "3"); + controller.run(); + + List files = controller.getFlowFilesForRelationship("success"); + + assertEquals("Process did not generate an output file", 1, files.size()); + + byte[] blob = files.get(0).toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File did not have 3 lines", 3, lines.length); + assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]); + } + + @Test + public void testSetupOptionalValidationR() { + LOG.info("Ruby script creating validators for optional properties"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.rb"); + controller.assertNotValid(); + controller.setProperty("int", "abc"); + controller.assertNotValid(); + controller.setProperty("url", "not@valid"); + controller.assertNotValid(); + controller.setProperty("nonEmpty", ""); + controller.assertNotValid(); + + controller.setProperty("int", "123"); + controller.setProperty("url", "http://localhost"); + controller.setProperty("nonEmpty", "abc123"); + controller.assertValid(); + } + + @Test + public void testTwoScriptsSameThreadSameClassName() { + LOG.info("Test 2 different scripts with the same ruby class name"); + + Map attrs1 = new HashMap<>(); + attrs1.put("filename", "StuffIsFine.txt"); + Map attrs2 = new HashMap<>(); + attrs2.put("filename", "multiline.txt"); + + controller.enqueue("This stuff is fine".getBytes(), attrs1); + controller.enqueue(multiline.getBytes(), attrs2); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb"); + controller.setProperty("expr", "rehenderit"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.rb"); + controller.assertNotValid(); + controller.setProperty("int", "abc"); + controller.assertNotValid(); + controller.setProperty("url", "not@valid"); + controller.assertNotValid(); + controller.setProperty("nonEmpty", ""); + controller.assertNotValid(); + + controller.setProperty("int", "123"); + controller.setProperty("url", "http://localhost"); + controller.setProperty("nonEmpty", "abc123"); + controller.assertValid(); + } + + @Test + public void testUpdateScriptR() throws Exception { + LOG.info("Test one script with updated class"); + + File testFile = File.createTempFile("script", ".rb"); + File original = new File("target/test-scripts/readWithParams.rb"); + FileUtils.copyFile(original, testFile); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath()); + controller.assertValid(); + original = new File("target/test-scripts/optionalValidators.rb"); + FileUtils.copyFile(original, testFile); + controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs"); + Thread.sleep(6000); + + controller.assertNotValid(); + controller.setProperty("int", "abc"); + controller.assertNotValid(); + controller.setProperty("url", "not@valid"); + controller.assertNotValid(); + controller.setProperty("nonEmpty", ""); + controller.assertNotValid(); + + controller.setProperty("int", "123"); + controller.setProperty("url", "http://localhost"); + controller.setProperty("nonEmpty", "abc123"); + controller.assertValid(); + FileUtils.deleteQuietly(testFile); + } + + @Test + public void testMultiThreadExecR() { + LOG.info("Ruby script 20 threads: Failing file based on reading contents"); + + Map attrs1 = new HashMap<>(); + attrs1.put("filename", "StuffIsFine.txt"); + Map attrs2 = new HashMap<>(); + attrs2.put("filename", "multiline.txt"); + controller.setThreadCount(20); + for (int i = 0; i < 10; i++) { + controller.enqueue("This stuff is fine".getBytes(), attrs1); + controller.enqueue(multiline.getBytes(), attrs2); + } + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.rb"); + controller.setProperty("expr", "rehenderit"); + controller.run(20); + + controller.assertTransferCount("failure", 10); + controller.assertTransferCount("success", 10); + for (int i = 0; i < 10; i++) { + MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i); + ff.assertContentEquals(multiline); + assertTrue(ff.getAttribute("filename").endsWith("modified")); + ff = controller.getFlowFilesForRelationship("success").get(i); + ff.assertContentEquals("This stuff is fine"); + assertTrue(ff.getAttribute("filename").endsWith("modified")); + } + + } + + @Test + public void testManualValidationR() { + LOG.info("Ruby script defining manual validator"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.rb"); + controller.assertNotValid(); + } + + @Test + public void testGetRelationshipsR() { + LOG.info("Ruby script: getRelationships"); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.rb"); + // at this point, the script has not been instantiated so the processor simply returns an empty set + Set rels = controller.getProcessor().getRelationships(); + assertEquals(0, rels.size()); + // this will instantiate the script + controller.assertValid(); + // this will call the script + rels = controller.getProcessor().getRelationships(); + assertEquals(3, rels.size()); + } + + @Test + public void testGetExceptionRouteR() { + LOG.info("Ruby script defining route taken in event of exception"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue("Bad things go to 'b'.".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.rb"); + + controller.run(3); + + assertRelationshipContents("This stuff is fine", "a"); + assertRelationshipContents("Bad things go to 'b'.", "b"); + assertRelationshipContents(multiline, "c"); + + } + + @Test + public void testSimpleConverterR() { + LOG.info("Running Ruby converter script"); + + for (int i = 0; i < 20; i++) { + controller.enqueue(multiline.getBytes()); + } + + controller.setThreadCount(20); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.rb"); + controller.run(20); + + List successFiles = controller.getFlowFilesForRelationship("success"); + List failFiles = controller.getFlowFilesForRelationship("failure"); + + assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size()); + assertEquals("Process did not generate 20 FAILURE files", 20, failFiles.size()); + + MockFlowFile sFile = successFiles.get(19); + MockFlowFile fFile = failFiles.get(19); + + byte[] blob = fFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File had more than one line", 1, lines.length); + assertEquals("Lorem ipsum dolor sit amet,", lines[0]); + + blob = sFile.toByteArray(); + lines = new String(blob).split("\n"); + + assertEquals("SUCCESS had wrong number of lines", 7, lines.length); + assertEquals("consectetur adipisicing elit,", lines[0]); + } + + @Test + public void testLoadLocalR() { + LOG.info("Ruby: load another script file"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.rb"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testFlowFileR() { + LOG.info("Ruby: get FlowFile properties"); + + controller.enqueue(multiline.getBytes()); + HashMap meta = new HashMap(); + meta.put("evict", "yup"); + controller.enqueue("This would be plenty long but it's also evicted.".getBytes(), meta); + controller.enqueue("This is too short".getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.rb"); + controller.run(3); + + assertRelationshipContents(multiline, "success"); + assertRelationshipContents("This is too short", "failure"); + assertRelationshipContents("This would be plenty long but it's also evicted.", "evict"); + } + + // //////////////////////////////////// // JS tests + @Test + public void testSimpleReadJS() { + LOG.info("Javascript: fail file based on reading contents"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.js"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testParamReadJS() { + LOG.info("Javascript: read contents and fail based on parameter"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.js"); + controller.setProperty("expr", "sed do"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testWriteLastLineJS() { + LOG.info("Running Javascript to output last line of file"); + + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.js"); + controller.run(); + + List sunkFiles = controller.getFlowFilesForRelationship("success"); + + assertEquals("Process did not generate an output file", 1, sunkFiles.size()); + + MockFlowFile sunkFile = sunkFiles.iterator().next(); + byte[] blob = sunkFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File had more than one line", 1, lines.length); + assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]); + } + + @Test + public void testWriteOptionalParametersJS() { + LOG.info("Javascript processCallback that uses optional parameters"); + + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.js"); + controller.setProperty("repeat", "3"); + controller.run(); + + List sunkFiles = controller.getFlowFilesForRelationship("success"); + + assertEquals("Process did not generate an output file", 1, sunkFiles.size()); + + MockFlowFile sunkFile = sunkFiles.iterator().next(); + byte[] blob = sunkFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File did not have 3 lines", 3, lines.length); + assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]); + } + + @Test + public void testSetupOptionalValidationJS() { + LOG.info("Javascript creating validators for optional properties"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.js"); + controller.setProperty("int", "abc"); + controller.setProperty("url", "not@valid"); + controller.setProperty("nonEmpty", ""); + assertEquals(2, controller.getProcessor().getPropertyDescriptors().size()); + controller.assertNotValid(); // due to invalid values above + assertEquals(5, controller.getProcessor().getPropertyDescriptors().size()); + + controller.setProperty("int", "123"); + controller.setProperty("url", "http://localhost"); + controller.setProperty("nonEmpty", "abc123"); + assertEquals(5, controller.getProcessor().getPropertyDescriptors().size()); + controller.assertValid(); + } + + @Test + public void testManualValidationJS() { + LOG.info("Javascript defining manual validator"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.js"); + controller.assertNotValid(); + } + + @Test + public void testGetExceptionRouteJS() { + LOG.info("Javascript defining route taken in event of exception"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue("Bad things go to 'b'.".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.js"); + + controller.run(3); + + assertRelationshipContents("This stuff is fine", "a"); + assertRelationshipContents("Bad things go to 'b'.", "b"); + assertRelationshipContents(multiline, "c"); + + } + + @Test + public void testSimpleConverterJS() { + LOG.info("Running Javascript converter script"); + + for (int i = 0; i < 20; i++) { + controller.enqueue(multiline.getBytes()); + } + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.js"); + controller.run(20); + + List successFiles = controller.getFlowFilesForRelationship("success"); + List failFiles = controller.getFlowFilesForRelationship("failure"); + + assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size()); + assertEquals("Process did not generate 20 FAILURE file", 20, failFiles.size()); + + MockFlowFile sFile = successFiles.get(19); + MockFlowFile fFile = failFiles.get(0); + + byte[] blob = sFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("SUCCESS had wrong number of lines", 7, lines.length); + assertTrue(lines[0].startsWith("consectetur adipisicing elit,")); + + blob = fFile.toByteArray(); + lines = new String(blob).split("\n"); + + assertEquals("File had more than one line", 1, lines.length); + assertTrue(lines[0].startsWith("Lorem ipsum dolor sit amet,")); + } + + @Test + public void testLoadLocalJS() { + LOG.info("Javascript: load another script file"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.js"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testXMLJS() { + LOG.info("Javascript: native XML parser"); + + controller.enqueue("BadGoodBad".getBytes()); + controller.enqueue("Helloworld".getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/parseXml.js"); + controller.run(2); + + assertRelationshipContents("Good", "success"); + assertRelationshipContents("Helloworld", "failure"); + } + + @Test + public void testFlowFileJS() { + LOG.info("JavaScript: get FlowFile properties"); + + controller.enqueue("This is too short".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.js"); + controller.run(2); + + assertRelationshipContents(multiline, "success"); + assertRelationshipContents("This is too short", "failure"); + } + + @Test + public void testMultiThreadExecJS() { + LOG.info("JavaScript script 20 threads: Failing file based on reading contents"); + + Map attrs1 = new HashMap<>(); + attrs1.put("filename", "StuffIsFine.txt"); + Map attrs2 = new HashMap<>(); + attrs2.put("filename", "multiline.txt"); + controller.setThreadCount(20); + for (int i = 0; i < 10; i++) { + controller.enqueue("This stuff is fine".getBytes(), attrs1); + controller.enqueue(multiline.getBytes(), attrs2); + } + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.js"); + controller.setProperty("expr", "rehenderit"); + controller.run(20); + + controller.assertTransferCount("failure", 10); + controller.assertTransferCount("success", 10); + for (int i = 0; i < 10; i++) { + MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i); + ff.assertContentEquals(multiline); + assertTrue(ff.getAttribute("filename").endsWith("modified")); + ff = controller.getFlowFilesForRelationship("success").get(i); + ff.assertContentEquals("This stuff is fine"); + assertTrue(ff.getAttribute("filename").endsWith("modified")); + } + } + + @Test + public void testUpdateScriptJS() throws Exception { + LOG.info("Test one script with updated class"); + + File testFile = File.createTempFile("script", ".js"); + File original = new File("target/test-scripts/readWithParams.js"); + FileUtils.copyFile(original, testFile); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath()); + controller.assertValid(); + original = new File("target/test-scripts/optionalValidators.js"); + FileUtils.copyFile(original, testFile); + controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs"); + Thread.sleep(6000); + + controller.assertNotValid(); + controller.setProperty("int", "abc"); + controller.assertNotValid(); + controller.setProperty("url", "not@valid"); + controller.assertNotValid(); + controller.setProperty("nonEmpty", ""); + controller.assertNotValid(); + + controller.setProperty("int", "123"); + controller.setProperty("url", "http://localhost"); + controller.setProperty("nonEmpty", "abc123"); + controller.assertValid(); + FileUtils.deleteQuietly(testFile); + } + + // ////////////////////////////////// // Python script tests + @Test + public void testSimpleReadP() { + LOG.info("Python script: fail file based on reading contents"); + + for (int i = 0; i < 20; i++) { + Map attr1 = new HashMap<>(); + attr1.put("filename", "FineStuff"); + attr1.put("counter", Integer.toString(i)); + Map attr2 = new HashMap<>(); + attr2.put("filename", "MultiLine"); + attr2.put("counter", Integer.toString(i)); + controller.enqueue("This stuff is fine".getBytes(), attr1); + controller.enqueue(multiline.getBytes(), attr2); + } + + controller.setThreadCount(40); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readTest.py"); + controller.run(40); + + assertRelationshipContents(multiline, "failure", 20); + assertRelationshipContents("This stuff is fine", "success", 20); + + List fails = controller.getFlowFilesForRelationship("failure"); + List successes = controller.getFlowFilesForRelationship("success"); + for (int i = 0; i < 20; i++) { + assertTrue(fails.get(i).getAttribute("filename").matches("^.*\\d+$")); + assertTrue(successes.get(i).getAttribute("filename").matches("^.*\\d+$")); + } + } + + @Test + public void testParamReadP() { + LOG.info("Python script: read contents and fail based on parameter"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.py"); + controller.setProperty("expr", "sed do"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testWriteLastLineP() { + LOG.info("Running Python script to output last line of file"); + + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/writeTest.py"); + controller.run(); + + List sunkFiles = controller.getFlowFilesForRelationship("success"); + + assertEquals("Process did not generate an output file", 1, sunkFiles.size()); + + MockFlowFile sunkFile = sunkFiles.iterator().next(); + byte[] blob = sunkFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File had more than one line", 1, lines.length); + assertEquals("sunt in culpa qui officia deserunt mollit anim id est laborum.", lines[0]); + } + + @Test + public void testWriteOptionalParametersP() { + LOG.info("Python script processCallback that uses optional parameters"); + + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/paramTest.py"); + controller.setProperty("repeat", "3"); + controller.run(); + + List sunkFiles = controller.getFlowFilesForRelationship("success"); + + assertEquals("Process did not generate an output file", 1, sunkFiles.size()); + + MockFlowFile sunkFile = sunkFiles.iterator().next(); + byte[] blob = sunkFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("File did not have 3 lines", 3, lines.length); + assertTrue(lines[2].startsWith("sunt in culpa qui officia deserunt mollit anim id est laborum.")); + } + + @Test + public void testManualValidationP() { + LOG.info("Python defining manual validator"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/alwaysFail.py"); + controller.assertNotValid(); + } + + @Test + public void testSetupOptionalValidationP() { + LOG.info("Python script creating validators for optional properties"); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/optionalValidators.py"); + controller.setProperty("int", "abc"); + controller.setProperty("url", "not@valid"); + controller.setProperty("nonEmpty", ""); + assertEquals(2, controller.getProcessor().getPropertyDescriptors().size()); + controller.assertNotValid(); + + controller.setProperty("int", "123"); + controller.setProperty("url", "http://localhost"); + controller.setProperty("nonEmpty", "abc123"); + assertEquals(5, controller.getProcessor().getPropertyDescriptors().size()); + controller.assertValid(); + } + + @Test + public void testGetExceptionRouteP() { + LOG.info("Python script defining route taken in event of exception"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue("Bad things go to 'b'.".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/routeTest.py"); + + // Don't put the error in the logs + // TestableAppender ta = new TestableAppender(); + // ta.attach(Logger.getLogger(ExecuteScript.class)); + controller.run(3); + // ta.detach(); + + assertRelationshipContents("This stuff is fine", "a"); + assertRelationshipContents("Bad things go to 'b'.", "b"); + assertRelationshipContents(multiline, "c"); + + // ta.assertFound("threw exception"); + } + + @Test + public void testLoadLocalP() throws Exception { + + final Thread t = new Thread(new Runnable() { + @Override + public void run() { + try { + testGetExceptionRouteP(); + setupEach(); + } catch (Exception e) { + + } + } + }); + + t.start(); + t.join(); + + LOG.info("Python: load another script file"); + + controller.enqueue("This stuff is fine".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/loadLocal.py"); + controller.run(2); + + assertRelationshipContents(multiline, "failure"); + assertRelationshipContents("This stuff is fine", "success"); + } + + @Test + public void testSimpleConverterP() { + LOG.info("Running Python converter script"); + + for (int i = 0; i < 20; i++) { + controller.enqueue(multiline.getBytes()); + } + + controller.setThreadCount(20); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/simpleConverter.py"); + controller.run(20); + + List successFiles = controller.getFlowFilesForRelationship("success"); + List failFiles = controller.getFlowFilesForRelationship("failure"); + + assertEquals("Process did not generate 20 SUCCESS files", 20, successFiles.size()); + assertEquals("Process did not generate 20 FAILURE files", 20, failFiles.size()); + + MockFlowFile sFile = successFiles.iterator().next(); + MockFlowFile fFile = failFiles.iterator().next(); + + byte[] blob = sFile.toByteArray(); + String[] lines = new String(blob).split("\n"); + + assertEquals("SUCCESS had wrong number of lines", 7, lines.length); + assertTrue(lines[0].startsWith("consectetur adipisicing elit,")); + + blob = fFile.toByteArray(); + lines = new String(blob).split("\n"); + + assertEquals("File had more than one line", 1, lines.length); + assertTrue(lines[0].startsWith("Lorem ipsum dolor sit amet,")); + } + + @Test + public void testFlowFileP() { + LOG.info("Python: get FlowFile properties"); + + controller.enqueue("This is too short".getBytes()); + controller.enqueue(multiline.getBytes()); + + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/ffTest.py"); + controller.run(2); + + assertRelationshipContents(multiline, "success"); + assertRelationshipContents("This is too short", "failure"); + } + + @Test + public void testMultiThreadExecP() { + LOG.info("Pthon script 20 threads: Failing file based on reading contents"); + + Map attrs1 = new HashMap<>(); + attrs1.put("filename", "StuffIsFine.txt"); + Map attrs2 = new HashMap<>(); + attrs2.put("filename", "multiline.txt"); + for (int i = 0; i < 10; i++) { + controller.enqueue("This stuff is fine".getBytes(), attrs1); + controller.enqueue(multiline.getBytes(), attrs2); + } + + controller.setThreadCount(20); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, "target/test-scripts/readWithParams.py"); + controller.setProperty("expr", "sed do"); + controller.run(20); + + controller.assertTransferCount("failure", 10); + controller.assertTransferCount("success", 10); + for (int i = 0; i < 10; i++) { + MockFlowFile ff = controller.getFlowFilesForRelationship("failure").get(i); + ff.assertContentEquals(multiline); + assertTrue(ff.getAttribute("filename").endsWith("modified")); + ff = controller.getFlowFilesForRelationship("success").get(i); + ff.assertContentEquals("This stuff is fine"); + assertTrue(ff.getAttribute("filename").endsWith("modified")); + } + } + + @Test + public void testUpdateScriptP() throws Exception { + LOG.info("Test one script with updated class"); + + File testFile = File.createTempFile("script", ".py"); + File original = new File("target/test-scripts/readTest.py"); + FileUtils.copyFile(original, testFile); + controller.setProperty(ExecuteScript.SCRIPT_FILE_NAME, testFile.getPath()); + controller.assertValid(); + original = new File("target/test-scripts/readWithParams.py"); + FileUtils.copyFile(original, testFile); + controller.setProperty(ExecuteScript.SCRIPT_CHECK_INTERVAL, "5 secs"); + Thread.sleep(6000); + + controller.assertNotValid(); // need to set 'expr' + controller.setProperty("int", "abc"); + controller.assertNotValid(); + controller.setProperty("url", "not@valid"); + controller.assertNotValid(); + controller.setProperty("nonEmpty", ""); + controller.assertNotValid(); + + controller.setProperty("expr", "sed do"); + controller.assertValid(); + assertEquals(6, controller.getProcessContext().getProperties().size()); + FileUtils.deleteQuietly(testFile); + } + +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js new file mode 100644 index 0000000000..40f038dcda --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.js @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var instance = new ReaderScript({ + validate: function () { + return ["This will never work."]; + } + }); +} + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py new file mode 100644 index 0000000000..4b744ab8db --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.py @@ -0,0 +1,19 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# type(name, superclass_tuple, member_dict) is a shorthand for defining an +# anonymous class. Note the trailing parens (), because scriptBuilder must +# return an *instance* of the class. +instance = type("FailingReader", (ReaderScript, object),\ + {"validate": lambda self : ["I am broken"]})() \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb new file mode 100644 index 0000000000..076cdfa498 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/alwaysFail.rb @@ -0,0 +1,21 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class NeverSatisfied < ReaderScript + def validate + return ["This is supposed to fail"] + end +end + +NeverSatisfied.new diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js new file mode 100644 index 0000000000..d40623e94e --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.js @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var instance = new ReaderScript({ + route: function (input) { + if (instance.getFlowFileSize() < 20) { + return Script.FAIL_RELATIONSHIP; + } else { + return Script.SUCCESS_RELATIONSHIP; + } + } + }); + +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py new file mode 100644 index 0000000000..f4deb227aa --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.py @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJythonReader(ReaderScript): + def route(self, input): + if self.getFlowFileSize() < 20 : return self.FAIL_RELATIONSHIP + + return self.SUCCESS_RELATIONSHIP + + +instance = SimpleJythonReader() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb new file mode 100644 index 0000000000..0435ea7d5f --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/ffTest.rb @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJRubyReader < ReaderScript + field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP + + def getRelationships + @@evict = Relationship::Builder.new.name("evict").description("some evicted stuff").build() + [FAIL_RELATIONSHIP, SUCCESS_RELATIONSHIP, @@evict] + end + + def route( input ) + return FAIL_RELATIONSHIP if getFlowFileSize < 20 + return @@evict if !getAttribute("evict").nil? + return SUCCESS_RELATIONSHIP + end +end + +SimpleJRubyReader.new diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py new file mode 100644 index 0000000000..827816cc54 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/Sub.py @@ -0,0 +1,18 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class TestFilter: + def notAllowed(self): + return "^sed" + \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js new file mode 100644 index 0000000000..489036a01c --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.js @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +function notAllowed() { // Works for eval(readFile(...)) + return /sed do/i; +} + +exports.notAllowed = notAllowed; // Works for require(...) + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb new file mode 100644 index 0000000000..775aad4f5b --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/lib/sub.rb @@ -0,0 +1,17 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +def notAllowed + return /^sed/i +end \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js new file mode 100644 index 0000000000..39396ab37d --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.js @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +var sub = require("lib/sub.js"); + +with (Scripting) { + var instance = new ReaderScript({ + route: function (input) { + var str = IOUtils.toString(input); + if (str.match(sub.notAllowed())) { + return Script.FAIL_RELATIONSHIP; + } else { + return Script.SUCCESS_RELATIONSHIP; + } + } + }); +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py new file mode 100644 index 0000000000..2429ea20ad --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.py @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import re +from Sub import TestFilter +class SimpleJythonReader(ReaderScript): + def route(self, input): + tf = TestFilter() + for line in FileUtil.wrap(input): + if re.match(tf.notAllowed(),line): return self.FAIL_RELATIONSHIP + + return self.SUCCESS_RELATIONSHIP + + +instance = SimpleJythonReader() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb new file mode 100644 index 0000000000..f1b3ec5c97 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/loadLocal.rb @@ -0,0 +1,29 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +require 'sub' + +class SimpleJRubyReader < ReaderScript + field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP + + def route( input ) + input.to_io.each_line do |line| + return FAIL_RELATIONSHIP if line.match notAllowed + end + + return SUCCESS_RELATIONSHIP + end +end + +SimpleJRubyReader.new diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml new file mode 100644 index 0000000000..d984dffb05 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/log4j.xml @@ -0,0 +1,54 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js new file mode 100644 index 0000000000..cf7c551ef4 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.js @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var instance = new WriterScript({ + getPropertyDescriptors: function () { + i = new PropertyDescriptor.Builder().name("int").description("an int").required(true).addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build(); + u = new PropertyDescriptor.Builder().name("url").description("a url").required(true).addValidator(StandardValidators.URL_VALIDATOR).build(); + s = new PropertyDescriptor.Builder().name("nonEmpty").description("a non empty property").required(true).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build(); + return [i, u, s]; + } + }); + +} + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py new file mode 100644 index 0000000000..9f1a43de7f --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.py @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJythonReader(ReaderScript): + def getPropertyDescriptors( self ): + nev = PropertyDescriptor.Builder().name("nonEmpty").required(1).addValidator(StandardValidators.NON_EMPTY_VALIDATOR).build() + iv = PropertyDescriptor.Builder().name("int").required(1).addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR).build() + uv = PropertyDescriptor.Builder().name("url").required(1).addValidator(StandardValidators.URL_VALIDATOR).build() + return [nev, iv, uv] + +instance = SimpleJythonReader() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb new file mode 100644 index 0000000000..3fa53b4c8b --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/optionalValidators.rb @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJRubyReader < ReaderScript + field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger + + def getPropertyDescriptors + logger.debug("Defining descriptors"); + i = StandardValidators::INTEGER_VALIDATOR + u = StandardValidators::URL_VALIDATOR + s = StandardValidators::NON_EMPTY_VALIDATOR + intPropDesc = PropertyDescriptor::Builder.new().name("int").required(true).addValidator(i).build() + urlPropDesc = PropertyDescriptor::Builder.new().name("url").required(true).addValidator(u).build() + nonEmptyPropDesc = PropertyDescriptor::Builder.new().name("nonEmpty").addValidator(s).build() + return [intPropDesc, urlPropDesc, nonEmptyPropDesc] + end + + def route( input ) + logger.debug("Routing input"); + input.to_io.each_line do |line| + return FAIL_RELATIONSHIP if line.match /^sed/i + end + + return SUCCESS_RELATIONSHIP + end +end +$logger.debug("Creating SimpleJRubyReader with props" + @properties.to_s) +SimpleJRubyReader.new \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js new file mode 100644 index 0000000000..c00537ab46 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.js @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var instance = new WriterScript({ + process: function (input, output) { + var str = IOUtils.toString(input); + var last = str.split("\n").pop() + "\n"; + for (var i = 0; i < instance.getProperty("repeat"); i++) { + IOUtils.write(last, output); + } + output.flush(); + } + }); +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py new file mode 100644 index 0000000000..3495e4a80d --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.py @@ -0,0 +1,26 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class ParamUsingWriter(WriterScript): + def process ( self, input, output ): + last = FileUtil.wrap(input).readlines()[-1] + '\n' + writer = FileUtil.wrap(output) + times = int(self.getProperty("repeat")) + lines = [last] * times + + writer.writelines(lines) + + writer.close() + +instance = ParamUsingWriter() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb new file mode 100644 index 0000000000..7c2eb23c7c --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/paramTest.rb @@ -0,0 +1,31 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class ParamUsingWriter < WriterScript + def process ( input, output ) + reader = input.to_io + writer = output.to_io + + last = reader.readlines.last + + getProperty("repeat").to_i.times do + writer << last + "\n" + end + + writer.close + reader.close + end +end + +ParamUsingWriter.new diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js new file mode 100644 index 0000000000..901e1589d2 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/parseXml.js @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var instance = new WriterScript({ + process: function (istream, ostream) { + var str = IOUtils.toString(istream); + var obj = new XML(str); + print(obj) + var good = obj.b.(@good == "true"); + if (good.length() == 0) { + instance.setRoute(Script.FAIL_RELATIONSHIP); + IOUtils.write(str, ostream); + } else { + instance.setRoute(Script.SUCCESS_RELATIONSHIP); + for each (var goodStr in good) { + IOUtils.write(goodStr, ostream); + } + } + ostream.flush(); + } + }); +} \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js new file mode 100644 index 0000000000..f5d3eccb18 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.js @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + + var instance = new ReaderScript({ + route: function (input) { + str = IOUtils.toString(input); + if (str.match(/sed do/i)) { + return Script.FAIL_RELATIONSHIP; + } else { + return Script.SUCCESS_RELATIONSHIP; + } + } + }); + logger.debug("Got a logger and properties" + properties); +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py new file mode 100644 index 0000000000..91e6ca7939 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.py @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os, re, subprocess + +class SimpleJythonReader(ReaderScript): + def route(self, input): + logger.info("In route") + returnid = os.system("c:\\cygwin\\bin\\echo GOOD") + fname = self.getAttribute("filename") + counter = self.getAttribute("counter") + fname = fname + '.' + counter + self.setAttribute("filename", fname) + for line in FileUtil.wrap(input): + if re.match("^sed",line): return self.FAIL_RELATIONSHIP + + return self.SUCCESS_RELATIONSHIP + + + +instance = SimpleJythonReader() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb new file mode 100644 index 0000000000..c10765d9d6 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readTest.rb @@ -0,0 +1,30 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJRubyReader < ReaderScript + field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes + + def route( input ) + logger.info("Route Input") + input.to_io.each_line do |line| + return FAIL_RELATIONSHIP if line.match /^sed/i + end + + attributes.put("filename", "NewFileNameFromReadTest") + return SUCCESS_RELATIONSHIP + end +end +$logger.info("Logger is made available in shared variables...however, the SimpleJRubyReader.logger is not set till after this script returns") + +SimpleJRubyReader.new diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js new file mode 100644 index 0000000000..e34ad5f70d --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.js @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var instance = new ReaderScript({ + route: function (input) { + var str = IOUtils.toString(input); + var expr = instance.getProperty("expr"); + filename = instance.attributes.get("filename"); + instance.setAttribute("filename", filename + ".modified"); + if (str.match(expr)) { + return Script.FAIL_RELATIONSHIP; + } else { + return Script.SUCCESS_RELATIONSHIP; + } + } + }); +} + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py new file mode 100644 index 0000000000..23e55af244 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.py @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import re + +class SimpleJythonReader(ReaderScript): + def getPropertyDescriptors( self ): + nev = StandardValidators.NON_EMPTY_VALIDATOR + return [PropertyDescriptor.Builder().name("expr").required(1).addValidator(nev).build()] + + def route( self, input ): + expr = self.getProperty("expr") + filename = self.getAttribute("filename") + self.setAttribute("filename", filename + ".modified") + for line in FileUtil.wrap(input): + if re.match(expr, line): return self.FAIL_RELATIONSHIP + + return self.SUCCESS_RELATIONSHIP + +instance = SimpleJythonReader() + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb new file mode 100644 index 0000000000..308b652e98 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/readWithParams.rb @@ -0,0 +1,33 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJRubyReader < ReaderScript + field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :properties, :attributes + + def route( input ) + expr = properties.get "expr" + raise "Must specify the 'expr' property!" if expr.nil? + filename = attributes.get "filename" + setAttribute("filename", filename + ".modified") + input.to_io.each_line do |line| + return FAIL_RELATIONSHIP if line.match expr + end + + return SUCCESS_RELATIONSHIP + end +end + +$logger.debug("Can access logger and properties via shared instance variables...props = " + @properties.to_s) +SimpleJRubyReader.new + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js new file mode 100644 index 0000000000..47ef546e92 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.js @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + var a = new Relationship.Builder().name("a").description("some good stuff").build() + var b = new Relationship.Builder().name("b").description("some other stuff").build() + var c = new Relationship.Builder().name("c").description("some bad stuff").build() + var instance = new ReaderScript({ + getExceptionRoute: function () { + return c; + }, + getRelationships: function () { + return [a, b, c]; + }, + route: function (input) { + var str = IOUtils.toString(input); + var lines = str.split("\n"); + for (var line in lines) { + if (lines[line].match(/^bad/i)) { + return b; + } else if (lines[line].match(/^sed/i)) { + throw "That's no good!"; + } + } + return a; + } + }); +} diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py new file mode 100644 index 0000000000..ed5408a7e2 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.py @@ -0,0 +1,37 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import re + +class RoutingReader(ReaderScript): + A = Relationship.Builder().name("a").description("some good stuff").build() + B = Relationship.Builder().name("b").description("some other stuff").build() + C = Relationship.Builder().name("c").description("some bad stuff").build() + + def getRelationships(self): + return [self.A,self.B,self.C] + + def getExceptionRoute(self): + return self.C + + def route( self, input ): + for line in FileUtil.wrap(input): + if re.match("^bad", line, re.IGNORECASE): + return self.B + if re.match("^sed", line): + raise RuntimeError("That's no good!") + + return self.A + +instance = RoutingReader() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb new file mode 100644 index 0000000000..38249ae1c2 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/routeTest.rb @@ -0,0 +1,39 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJRubyReader < ReaderScript + + @@a = Relationship::Builder.new().name("a").description("some good stuff").build() + @@b = Relationship::Builder.new().name("b").description("some bad stuff").build() + @@c = Relationship::Builder.new().name("c").description("some other stuff").build() + + def getRelationships + return [@@a, @@b, @@c] + end + + def getExceptionRoute + @@c + end + + def route( input ) + input.to_io.each_line do |line| + return @@b if line.match /^bad/i + raise "That's no good!" if line.match /^sed/i + end + + @@a + end +end + +SimpleJRubyReader.new diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js new file mode 100644 index 0000000000..72bb80e380 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.js @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +with (Scripting) { + + var instance = new ConverterScript({ + convert: function (input) { + var buffReader = new java.io.BufferedReader(new java.io.InputStreamReader(input)); + instance.createFlowFile("firstLine", Script.FAIL_RELATIONSHIP, function (output) { + var out = new java.io.BufferedWriter(new java.io.OutputStreamWriter(output)); + var firstLine = buffReader.readLine(); + out.write(firstLine, 0, firstLine.length()); + out.flush(); + out.close(); + }); + + instance.createFlowFile("otherLines", Script.SUCCESS_RELATIONSHIP, function (output) { + var out = new java.io.BufferedWriter(new java.io.OutputStreamWriter(output)); + var line = buffReader.readLine(); + while (line != null) { + out.write(line, 0, line.length()); + out.newLine(); + line = buffReader.readLine(); + } + out.flush(); + out.close(); + }); + } + + }); + logger.debug("Processor props" + properties) +} \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py new file mode 100644 index 0000000000..d52368e44c --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.py @@ -0,0 +1,60 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +from org.python.core.io import TextIOWrapper,BufferedReader,BufferedWriter,StreamIO +from org.apache.nifi.scripting import OutputStreamHandler + +class WriteFirstLine(OutputStreamHandler): + def __init__(self, wrappedIn): + self.wrappedIn = wrappedIn + + def write(self, output): + streamOut = StreamIO(output, False) + bufWrtr = BufferedWriter(streamOut, 8192) + wrappedOut = TextIOWrapper(bufWrtr) + wrappedOut.write(self.wrappedIn.readline(8192)) + wrappedOut.flush() + wrappedOut.close() + +class WriteOtherLines(OutputStreamHandler): + def __init__(self, wrappedIn): + self.wrappedIn = wrappedIn + + def write(self, output): + streamOut = StreamIO(output, False) + bufWrtr = BufferedWriter(streamOut, 8192) + wrappedOut = TextIOWrapper(bufWrtr) + line = self.wrappedIn.readline(8192) + while line != '': + wrappedOut.write(line) + line = self.wrappedIn.readline(8192) + wrappedOut.flush() + wrappedOut.close() + +class SimpleConverter(ConverterScript): + + def convert(self, input): + streamIn = StreamIO(input, False) + bufRdr = BufferedReader(streamIn, 8192) + wrappedIn = TextIOWrapper(bufRdr) + + writeFirstLine = WriteFirstLine(wrappedIn) + self.createFlowFile("firstLine", self.FAIL_RELATIONSHIP, writeFirstLine) + + writeOtherLines = WriteOtherLines(wrappedIn) + self.createFlowFile("otherLines", self.SUCCESS_RELATIONSHIP, writeOtherLines) + +instance = SimpleConverter() + + \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb new file mode 100644 index 0000000000..f33329489c --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/simpleConverter.rb @@ -0,0 +1,42 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +java_import 'org.apache.nifi.scripting.OutputStreamHandler' +class SimpleConverter < ConverterScript + field_reader :FAIL_RELATIONSHIP, :SUCCESS_RELATIONSHIP, :logger, :attributes + + def convert(input) + in_io = input.to_io + createFlowFile("firstLine", FAIL_RELATIONSHIP, OutputStreamHandler.impl do |method, out| + out_io = out.to_io + out_io << in_io.readline.to_java_bytes + out_io.close + logger.debug("Wrote data to failure...this message logged with logger from super class") + end) + + createFlowFile("otherLines", SUCCESS_RELATIONSHIP, OutputStreamHandler.impl do |method, out| + out_io = out.to_io + in_io.each_line { |line| + out_io << line + } + out_io.close + logger.debug("Wrote data to success...this message logged with logger from super class") + end) + in_io.close + end + +end + +$logger.debug("Creating SimpleConverter...this message logged with logger from shared variables") +SimpleConverter.new \ No newline at end of file diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js new file mode 100644 index 0000000000..1e055b7df4 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.js @@ -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. + */ +with (Scripting) { + var instance = new WriterScript({ + process: function (input, output) { + var str = IOUtils.toString(input); + IOUtils.write(str.split("\n").pop(), output); + output.flush(); + } + }); +} + diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py new file mode 100644 index 0000000000..c563b6686f --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.py @@ -0,0 +1,22 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleWriter(WriterScript): + def process( self, input, output ): + last = FileUtil.wrap(input).readlines()[-1] + writer = FileUtil.wrap(output) + writer.write(last) + writer.close() + +instance = SimpleWriter() diff --git a/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb new file mode 100644 index 0000000000..5be4553872 --- /dev/null +++ b/nar-bundles/execute-script-bundle/execute-script-processors/src/test/resources/writeTest.rb @@ -0,0 +1,32 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +class SimpleJRubyRunner < WriterScript + def process( input, output ) + in_io = input.to_io + out_io = output.to_io + + last = nil + in_io.each_line do |line| + last = line + end + + out_io << last unless last.nil? + + in_io.close + out_io.close + end +end + +SimpleJRubyRunner.new diff --git a/nar-bundles/execute-script-bundle/nar/pom.xml b/nar-bundles/execute-script-bundle/nar/pom.xml new file mode 100644 index 0000000000..1cc28dc7b2 --- /dev/null +++ b/nar-bundles/execute-script-bundle/nar/pom.xml @@ -0,0 +1,36 @@ + + + 4.0.0 + + + org.apache.nifi + execute-script-bundle + 0.0.1-SNAPSHOT + + + execute-script-nar + Execute Script NAR + nar + NiFi Script Running NAR + + + + org.apache.nifi + execute-script-processors + ${project.version} + + + diff --git a/nar-bundles/execute-script-bundle/pom.xml b/nar-bundles/execute-script-bundle/pom.xml new file mode 100644 index 0000000000..b3abb36afa --- /dev/null +++ b/nar-bundles/execute-script-bundle/pom.xml @@ -0,0 +1,81 @@ + + + 4.0.0 + + org.apache.nifi + nar-container-common + 0.0.1-SNAPSHOT + + + execute-script-bundle + 0.0.1-SNAPSHOT + + Execute Script Bundle + pom + + + + + org.apache.nifi + nifi-processor-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-stream-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-utils + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-core-flowfile-attributes + 0.0.1-SNAPSHOT + + + org.apache.nifi + nifi-mock + 0.0.1-SNAPSHOT + test + + + + org.jruby + jruby + 1.7.16.1 + + + + org.python + jython-standalone + 2.7-b3 + + + commons-io + commons-io + 2.4 + + + + + + execute-script-processors + nar + + diff --git a/nar-bundles/framework-bundle/framework/administration/.gitignore b/nar-bundles/framework-bundle/framework/administration/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nar-bundles/framework-bundle/framework/administration/pom.xml b/nar-bundles/framework-bundle/framework/administration/pom.xml new file mode 100644 index 0000000000..b5dd17170f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/pom.xml @@ -0,0 +1,116 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + + nifi-administration + 0.0.1-SNAPSHOT + NiFi Administration + + + + src/main/resources + + + src/main/xsd + + + + + org.codehaus.mojo + jaxb2-maven-plugin + + + current + + xjc + + + org.apache.nifi.authorization.generated + + + + + + + + + org.apache.nifi + nifi-user-actions + + + org.apache.nifi + nifi-api + + + org.apache.nifi + core-api + + + org.apache.nifi + nifi-nar + + + org.apache.nifi + nifi-utils + + + org.apache.nifi + nifi-properties + + + org.apache.nifi + nifi-security-utils + + + com.h2database + h2 + + + org.springframework + spring-beans + + + org.springframework + spring-context + + + org.springframework + spring-core + + + org.springframework + spring-aop + + + org.aspectj + aspectjweaver + + + org.springframework.security + spring-security-core + + + org.apache.commons + commons-collections4 + + + diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java new file mode 100644 index 0000000000..aeb2755276 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/AuditDataSourceFactoryBean.java @@ -0,0 +1,222 @@ +/* + * 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.admin; + +import java.io.File; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import org.apache.commons.lang3.StringUtils; +import org.h2.jdbcx.JdbcConnectionPool; +import org.apache.nifi.util.NiFiProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.FactoryBean; + +/** + * + */ +public class AuditDataSourceFactoryBean implements FactoryBean { + + private static final Logger logger = LoggerFactory.getLogger(AuditDataSourceFactoryBean.class); + private static final String NF_USERNAME_PASSWORD = "nf"; + private static final int MAX_CONNECTIONS = 5; + + // database file name + private static final String AUDIT_DATABASE_FILE_NAME = "nifi-audit"; + + // ------------ + // action table + // ------------ + private static final String CREATE_ACTION_TABLE = "CREATE TABLE ACTION (" + + "ID INT NOT NULL PRIMARY KEY AUTO_INCREMENT, " + + "USER_DN VARCHAR2(255) NOT NULL, " + + "USER_NAME VARCHAR2(100) NOT NULL, " + + "SOURCE_ID VARCHAR2(100) NOT NULL, " + + "SOURCE_NAME VARCHAR2(1000) NOT NULL, " + + "SOURCE_TYPE VARCHAR2(1000) NOT NULL, " + + "OPERATION VARCHAR2(50) NOT NULL, " + + "ACTION_TIMESTAMP TIMESTAMP NOT NULL " + + ")"; + + // ----------------- + // component details + // ----------------- + private static final String CREATE_PROCESSOR_DETAILS_TABLE = "CREATE TABLE PROCESSOR_DETAILS (" + + "ACTION_ID INT NOT NULL PRIMARY KEY, " + + "TYPE VARCHAR2(1000) NOT NULL, " + + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" + + ")"; + + private static final String CREATE_REMOTE_PROCESS_GROUP_DETAILS_TABLE = "CREATE TABLE REMOTE_PROCESS_GROUP_DETAILS (" + + "ACTION_ID INT NOT NULL PRIMARY KEY, " + + "URI VARCHAR2(2500) NOT NULL, " + + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" + + ")"; + + // -------------- + // action details + // -------------- + private static final String CREATE_MOVE_DETAILS_TABLE = "CREATE TABLE MOVE_DETAILS (" + + "ACTION_ID INT NOT NULL PRIMARY KEY, " + + "GROUP_ID VARCHAR2(100) NOT NULL, " + + "GROUP_NAME VARCHAR2(1000) NOT NULL, " + + "PREVIOUS_GROUP_ID VARCHAR2(100) NOT NULL, " + + "PREVIOUS_GROUP_NAME VARCHAR2(1000) NOT NULL, " + + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" + + ")"; + + private static final String CREATE_CONFIGURE_DETAILS_TABLE = "CREATE TABLE CONFIGURE_DETAILS (" + + "ACTION_ID INT NOT NULL PRIMARY KEY, " + + "NAME VARCHAR2(1000) NOT NULL, " + + "VALUE VARCHAR2(5000), " + + "PREVIOUS_VALUE VARCHAR2(5000), " + + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" + + ")"; + + private static final String CREATE_CONNECT_DETAILS_TABLE = "CREATE TABLE CONNECT_DETAILS (" + + "ACTION_ID INT NOT NULL PRIMARY KEY, " + + "SOURCE_ID VARCHAR2(100) NOT NULL, " + + "SOURCE_NAME VARCHAR2(1000), " + + "SOURCE_TYPE VARCHAR2(1000) NOT NULL, " + + "RELATIONSHIP VARCHAR2(1000), " + + "DESTINATION_ID VARCHAR2(100) NOT NULL, " + + "DESTINATION_NAME VARCHAR2(1000), " + + "DESTINATION_TYPE VARCHAR2(1000) NOT NULL, " + + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" + + ")"; + + private static final String CREATE_PURGE_DETAILS_TABLE = "CREATE TABLE PURGE_DETAILS (" + + "ACTION_ID INT NOT NULL PRIMARY KEY, " + + "END_DATE TIMESTAMP NOT NULL, " + + "FOREIGN KEY (ACTION_ID) REFERENCES ACTION(ID)" + + ")"; + + private JdbcConnectionPool connectionPool; + + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (connectionPool == null) { + + // locate the repository directory + String repositoryDirectoryPath = properties.getProperty(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY); + + // ensure the repository directory is specified + if (repositoryDirectoryPath == null) { + throw new NullPointerException("Database directory must be specified."); + } + + // create a handle to the repository directory + File repositoryDirectory = new File(repositoryDirectoryPath); + + // get a handle to the database file + File databaseFile = new File(repositoryDirectory, AUDIT_DATABASE_FILE_NAME); + + // format the database url + String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3"; + String databaseUrlAppend = properties.getProperty(NiFiProperties.H2_URL_APPEND); + if (StringUtils.isNotBlank(databaseUrlAppend)) { + databaseUrl += databaseUrlAppend; + } + + // create the pool + connectionPool = JdbcConnectionPool.create(databaseUrl, NF_USERNAME_PASSWORD, NF_USERNAME_PASSWORD); + connectionPool.setMaxConnections(MAX_CONNECTIONS); + + Connection connection = null; + ResultSet rs = null; + Statement statement = null; + try { + // get a connection + connection = connectionPool.getConnection(); + connection.setAutoCommit(false); + + // determine if the tables need to be created + rs = connection.getMetaData().getTables(null, null, "ACTION", null); + if (!rs.next()) { + logger.info("Database not built for repository: " + databaseUrl + ". Building now..."); + RepositoryUtils.closeQuietly(rs); + + // create a statement for initializing the database + statement = connection.createStatement(); + + // action table + statement.execute(CREATE_ACTION_TABLE); + + // component details + statement.execute(CREATE_PROCESSOR_DETAILS_TABLE); + statement.execute(CREATE_REMOTE_PROCESS_GROUP_DETAILS_TABLE); + + // action details + statement.execute(CREATE_MOVE_DETAILS_TABLE); + statement.execute(CREATE_CONFIGURE_DETAILS_TABLE); + statement.execute(CREATE_CONNECT_DETAILS_TABLE); + statement.execute(CREATE_PURGE_DETAILS_TABLE); + } else { + logger.info("Existing database found and connected to at: " + databaseUrl); + } + + // commit any changes + connection.commit(); + } catch (SQLException sqle) { + RepositoryUtils.rollback(connection, logger); + throw sqle; + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + RepositoryUtils.closeQuietly(connection); + } + } + + return connectionPool; + } + + @Override + public Class getObjectType() { + return JdbcConnectionPool.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + /** + * Disposes resources. + */ + public void shutdown() { + + // shutdown the connection pool + if (connectionPool != null) { + try { + connectionPool.dispose(); + } catch (Exception e) { + logger.warn("Unable to dispose of connection pool: " + e.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, e); + } + } + } + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java new file mode 100644 index 0000000000..b95388b821 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/RepositoryUtils.java @@ -0,0 +1,91 @@ +/* + * 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.admin; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import org.slf4j.Logger; + +/** + * A utility class for useful methods dealing with the repository + * + * @author unattributed + */ +public class RepositoryUtils { + + public static void rollback(final Connection conn, final Logger logger) { + try { + if (null != conn) { + conn.rollback(); + } + } catch (final SQLException sqe) { + logger.warn("The following problem occurred while trying to rollback " + conn + ": " + sqe.getLocalizedMessage()); + if (logger.isDebugEnabled()) { + logger.debug("", sqe); + } + } + } + + /** + * Closes the given statement quietly - no logging, no exceptions + * + * @param statement + */ + public static void closeQuietly(final Statement statement) { + + if (null != statement) { + try { + statement.close(); + } catch (final SQLException se) { /*IGNORE*/ + + } + } + } + + /** + * Closes the given result set quietly - no logging, no exceptions + * + * @param resultSet + */ + public static void closeQuietly(final ResultSet resultSet) { + if (null != resultSet) { + try { + resultSet.close(); + } catch (final SQLException se) {/*IGNORE*/ + + } + } + } + + /** + * Closes the given connection quietly - no logging, no exceptions + * + * @param conn + */ + public static void closeQuietly(final Connection conn) { + if (null != conn) { + try { + conn.close(); + } catch (final SQLException se) {/*IGNORE*/ + + } + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java new file mode 100644 index 0000000000..1f64f6eba2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/UserDataSourceFactoryBean.java @@ -0,0 +1,247 @@ +/* + * 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.admin; + +import java.io.File; +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.UUID; +import org.apache.commons.lang3.StringUtils; +import org.h2.jdbcx.JdbcConnectionPool; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.util.NiFiProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.factory.FactoryBean; + +/** + * + */ +public class UserDataSourceFactoryBean implements FactoryBean { + + private static final Logger logger = LoggerFactory.getLogger(UserDataSourceFactoryBean.class); + private static final String NF_USERNAME_PASSWORD = "nf"; + private static final int MAX_CONNECTIONS = 5; + + // database file name + private static final String AUDIT_DATABASE_FILE_NAME = "nifi-users"; + + private static final String CREATE_USER_TABLE = "CREATE TABLE USER (" + + "ID VARCHAR2(100) NOT NULL PRIMARY KEY, " + + "DN VARCHAR2(255) NOT NULL UNIQUE, " + + "USER_NAME VARCHAR2(100) NOT NULL, " + + "USER_GROUP VARCHAR2(100), " + + "CREATION TIMESTAMP NOT NULL, " + + "LAST_ACCESSED TIMESTAMP, " + + "LAST_VERIFIED TIMESTAMP, " + + "JUSTIFICATION VARCHAR2(500) NOT NULL, " + + "STATUS VARCHAR2(10) NOT NULL" + + ")"; + + private static final String CREATE_AUTHORITY_TABLE = "CREATE TABLE AUTHORITY (" + + "ID INT NOT NULL PRIMARY KEY AUTO_INCREMENT, " + + "USER_ID VARCHAR2(100) NOT NULL, " + + "ROLE VARCHAR2(50) NOT NULL, " + + "FOREIGN KEY (USER_ID) REFERENCES USER (ID), " + + "CONSTRAINT USER_ROLE_UNIQUE_CONSTRAINT UNIQUE (USER_ID, ROLE)" + + ")"; + + private static final String INSERT_ANONYMOUS_USER = "INSERT INTO USER (" + + "ID, DN, USER_NAME, CREATION, LAST_VERIFIED, JUSTIFICATION, STATUS" + + ") VALUES (" + + "'" + UUID.randomUUID().toString() + "', " + + "'" + NiFiUser.ANONYMOUS_USER_DN + "', " + + "'" + NiFiUser.ANONYMOUS_USER_DN + "', " + + "NOW(), " + + "NOW(), " + + "'Anonymous user needs no justification', " + + "'ACTIVE'" + + ")"; + + private static final String INSERT_ANONYMOUS_MONITOR_AUTHORITY = "INSERT INTO AUTHORITY (" + + "USER_ID, ROLE" + + ") VALUES (" + + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), " + + "'ROLE_MONITOR'" + + ")"; + + private static final String INSERT_ANONYMOUS_DFM_AUTHORITY = "INSERT INTO AUTHORITY (" + + "USER_ID, ROLE" + + ") VALUES (" + + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), " + + "'ROLE_DFM'" + + ")"; + + private static final String INSERT_ANONYMOUS_ADMIN_AUTHORITY = "INSERT INTO AUTHORITY (" + + "USER_ID, ROLE" + + ") VALUES (" + + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), " + + "'ROLE_ADMIN'" + + ")"; + + private static final String INSERT_ANONYMOUS_NIFI_AUTHORITY = "INSERT INTO AUTHORITY (" + + "USER_ID, ROLE" + + ") VALUES (" + + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), " + + "'ROLE_NIFI'" + + ")"; + + private static final String INSERT_ANONYMOUS_PROVENANCE_AUTHORITY = "INSERT INTO AUTHORITY (" + + "USER_ID, ROLE" + + ") VALUES (" + + "(SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "'), " + + "'ROLE_PROVENANCE'" + + ")"; + + private static final String SELECT_ANONYMOUS_PROVENANCE_AUTHORITY = "SELECT * FROM AUTHORITY " + + "WHERE " + + "USER_ID = (SELECT ID FROM USER WHERE DN = '" + NiFiUser.ANONYMOUS_USER_DN + "') " + + "AND " + + "ROLE = 'ROLE_PROVENANCE'"; + + private JdbcConnectionPool connectionPool; + + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (connectionPool == null) { + + // locate the repository directory + String repositoryDirectoryPath = properties.getProperty(NiFiProperties.REPOSITORY_DATABASE_DIRECTORY); + + // ensure the repository directory is specified + if (repositoryDirectoryPath == null) { + throw new NullPointerException("Database directory must be specified."); + } + + // create a handle to the repository directory + File repositoryDirectory = new File(repositoryDirectoryPath); + + // create a handle to the database directory and file + File databaseFile = new File(repositoryDirectory, AUDIT_DATABASE_FILE_NAME); + String databaseUrl = getDatabaseUrl(databaseFile); + + // create the pool + connectionPool = JdbcConnectionPool.create(databaseUrl, NF_USERNAME_PASSWORD, NF_USERNAME_PASSWORD); + connectionPool.setMaxConnections(MAX_CONNECTIONS); + + Connection connection = null; + ResultSet rs = null; + Statement statement = null; + try { + // get a connection + connection = connectionPool.getConnection(); + connection.setAutoCommit(false); + + // create a statement for creating/updating the database + statement = connection.createStatement(); + + // determine if the tables need to be created + rs = connection.getMetaData().getTables(null, null, "USER", null); + if (!rs.next()) { + logger.info("Database not built for repository: " + databaseUrl + ". Building now..."); + + // create the tables + statement.execute(CREATE_USER_TABLE); + statement.execute(CREATE_AUTHORITY_TABLE); + + // seed the anonymous user + statement.execute(INSERT_ANONYMOUS_USER); + statement.execute(INSERT_ANONYMOUS_MONITOR_AUTHORITY); + statement.execute(INSERT_ANONYMOUS_DFM_AUTHORITY); + statement.execute(INSERT_ANONYMOUS_ADMIN_AUTHORITY); + statement.execute(INSERT_ANONYMOUS_NIFI_AUTHORITY); + } else { + logger.info("Existing database found and connected to at: " + databaseUrl); + } + + // close the previous result set + RepositoryUtils.closeQuietly(rs); + + // merge in the provenance role to handle existing databases + rs = statement.executeQuery(SELECT_ANONYMOUS_PROVENANCE_AUTHORITY); + if (!rs.next()) { + statement.execute(INSERT_ANONYMOUS_PROVENANCE_AUTHORITY); + } + + // commit any changes + connection.commit(); + } catch (SQLException sqle) { + RepositoryUtils.rollback(connection, logger); + throw sqle; + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + RepositoryUtils.closeQuietly(connection); + } + } + + return connectionPool; + } + + /** + * Get the database url for the specified database file. + * + * @param databaseFile + * @return + */ + private String getDatabaseUrl(File databaseFile) { + String databaseUrl = "jdbc:h2:" + databaseFile + ";AUTOCOMMIT=OFF;DB_CLOSE_ON_EXIT=FALSE;LOCK_MODE=3"; + String databaseUrlAppend = properties.getProperty(NiFiProperties.H2_URL_APPEND); + if (StringUtils.isNotBlank(databaseUrlAppend)) { + databaseUrl += databaseUrlAppend; + } + return databaseUrl; + } + + @Override + public Class getObjectType() { + return JdbcConnectionPool.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + /** + * Disposes resources. + */ + public void shutdown() { + + // shutdown the connection pool + if (connectionPool != null) { + try { + connectionPool.dispose(); + } catch (Exception e) { + logger.warn("Unable to dispose of connection pool: " + e.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, e); + } + } + } + + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java new file mode 100644 index 0000000000..5d6d2226d8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/ActionDAO.java @@ -0,0 +1,74 @@ +/* + * 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.admin.dao; + +import java.util.Date; +import java.util.List; +import java.util.Map; +import org.apache.nifi.action.Action; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.History; +import org.apache.nifi.history.PreviousValue; + +/** + * Action data access. + */ +public interface ActionDAO { + + /** + * Persists the specified action. + * + * @param action + * @throws DataAccessException + */ + void createAction(Action action) throws DataAccessException; + + /** + * Finds all actions that meet the specified criteria. + * + * @param actionQuery + * @return + * @throws DataAccessException + */ + History findActions(HistoryQuery actionQuery) throws DataAccessException; + + /** + * Finds the previous values for the specified property in the specified + * processor. Returns empty list if there are none. + * + * @param processorId + * @return + */ + Map> getPreviousValues(String processorId); + + /** + * Finds the specified action. + * + * @param actionId + * @return + * @throws DataAccessException + */ + Action getAction(Integer actionId) throws DataAccessException; + + /** + * Deletes all actions up to the specified end date. + * + * @param endDate + * @throws DataAccessException + */ + void deleteActions(Date endDate) throws DataAccessException; +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java new file mode 100644 index 0000000000..2992884cbf --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/AuthorityDAO.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.dao; + +import java.util.Set; +import org.apache.nifi.authorization.Authority; + +/** + * Authority data access. + */ +public interface AuthorityDAO { + + /** + * Finds all Authority for the specified user. + * + * @param userId + * @return + */ + Set findAuthoritiesByUserId(String userId) throws DataAccessException; + + /** + * Creates a new Authorities for the specified user. + * + * @param authorities + * @param userId + */ + void createAuthorities(Set authorities, String userId) throws DataAccessException; + + /** + * Removes all Authorities for the specified user. + * + * @param userId + * @throws DataAccessException + */ + void deleteAuthorities(String userId) throws DataAccessException; + + /** + * Removes the specified Authority. + * + * @param authorities + * @param userId + */ + void deleteAuthorities(Set authorities, String userId) throws DataAccessException; +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java new file mode 100644 index 0000000000..dee4ef916d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DAOFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.dao; + +/** + * + */ +public interface DAOFactory { + + UserDAO getUserDAO(); + + ActionDAO getActionDAO(); + + AuthorityDAO getAuthorityDAO(); +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java new file mode 100644 index 0000000000..05bf4afa49 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/DataAccessException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.dao; + +/** + * Represents any error that might occur while administering NiFi accounts. + */ +public class DataAccessException extends RuntimeException { + + public DataAccessException(Throwable cause) { + super(cause); + } + + public DataAccessException(String message, Throwable cause) { + super(message, cause); + } + + public DataAccessException(String message) { + super(message); + } + + public DataAccessException() { + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java new file mode 100644 index 0000000000..9ffab5d8e3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/UserDAO.java @@ -0,0 +1,127 @@ +/* + * 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.admin.dao; + +import java.util.Date; +import java.util.Set; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; + +/** + * Defines the user data access object. + */ +public interface UserDAO { + + /** + * Determines whether there are any PENDING user accounts. + * + * @return + * @throws DataAccessException + */ + Boolean hasPendingUserAccounts() throws DataAccessException; + + /** + * Returns all users. + * + * @return + * @throws DataAccessException + */ + Set findUsers() throws DataAccessException; + + /** + * Returns all user groups. + * + * @return + * @throws DataAccessException + */ + Set findUserGroups() throws DataAccessException; + + /** + * Returns all users for the specified group. + * + * @param group + * @return + * @throws DataAccessException + */ + Set findUsersForGroup(String group) throws DataAccessException; + + /** + * Returns the user with the specified id. + * + * @param id + * @return + * @throws DataAccessException + */ + NiFiUser findUserById(String id) throws DataAccessException; + + /** + * Returns the user with the specified DN. + * + * @param dn + * @return + */ + NiFiUser findUserByDn(String dn) throws DataAccessException; + + /** + * Creates a new user based off the specified NiFiUser. + * + * @param user + */ + void createUser(NiFiUser user) throws DataAccessException; + + /** + * Updates the specified NiFiUser. + * + * @param user + */ + void updateUser(NiFiUser user) throws DataAccessException; + + /** + * Deletes the specified user. + * + * @param id + * @throws DataAccessException + */ + void deleteUser(String id) throws DataAccessException; + + /** + * Sets the status of the specified group. + * + * @param group + * @param status + * @throws DataAccessException + */ + void updateGroupStatus(String group, AccountStatus status) throws DataAccessException; + + /** + * Sets the last verified time for all users in the specified group. + * + * @param group + * @param lastVerified + * @throws DataAccessException S + */ + void updateGroupVerification(String group, Date lastVerified) throws DataAccessException; + + /** + * Ungroups the specified group. + * + * @param group + * @throws DataAccessException + */ + void ungroup(String group) throws DataAccessException; + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java new file mode 100644 index 0000000000..2f3de0edff --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/DAOFactoryImpl.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.dao.impl; + +import java.sql.Connection; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.UserDAO; + +/** + * + */ +public class DAOFactoryImpl implements DAOFactory { + + private final Connection connection; + + public DAOFactoryImpl(Connection connection) { + this.connection = connection; + } + + @Override + public ActionDAO getActionDAO() { + return new StandardActionDAO(connection); + } + + @Override + public AuthorityDAO getAuthorityDAO() { + return new StandardAuthorityDAO(connection); + } + + @Override + public UserDAO getUserDAO() { + return new StandardUserDAO(connection); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java new file mode 100644 index 0000000000..4b89655c47 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardActionDAO.java @@ -0,0 +1,1056 @@ +/* + * 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.admin.dao.impl; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import org.apache.nifi.action.Action; +import org.apache.nifi.action.Component; +import org.apache.nifi.action.Operation; +import org.apache.nifi.action.component.details.ComponentDetails; +import org.apache.nifi.action.component.details.ProcessorDetails; +import org.apache.nifi.action.component.details.RemoteProcessGroupDetails; +import org.apache.nifi.action.details.ActionDetails; +import org.apache.nifi.action.details.ConfigureDetails; +import org.apache.nifi.action.details.ConnectDetails; +import org.apache.nifi.action.details.MoveDetails; +import org.apache.nifi.action.details.PurgeDetails; +import org.apache.nifi.admin.RepositoryUtils; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; +import org.apache.commons.lang3.StringUtils; + +/** + * + */ +public class StandardActionDAO implements ActionDAO { + + // ------------ + // action table + // ------------ + private static final String INSERT_ACTION = "INSERT INTO ACTION (" + + "USER_DN, USER_NAME, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, OPERATION, ACTION_TIMESTAMP" + + ") VALUES (" + + "?, " + + "?, " + + "?, " + + "?, " + + "?, " + + "?, " + + "?, " + + ")"; + + // ----------------- + // component details + // ----------------- + private static final String INSERT_PROCESSOR_DETAILS = "INSERT INTO PROCESSOR_DETAILS (" + + "ACTION_ID, TYPE" + + ") VALUES (" + + "?, " + + "?" + + ")"; + + private static final String INSERT_REMOTE_PROCESS_GROUP_DETAILS = "INSERT INTO REMOTE_PROCESS_GROUP_DETAILS (" + + "ACTION_ID, URI" + + ") VALUES (" + + "?, " + + "?" + + ")"; + + // -------------- + // action details + // -------------- + private static final String INSERT_CONFIGURE_DETAILS = "INSERT INTO CONFIGURE_DETAILS (" + + "ACTION_ID, NAME, VALUE, PREVIOUS_VALUE" + + ") VALUES (" + + "?, " + + "?, " + + "?, " + + "?" + + ")"; + + private static final String INSERT_CONNECT_DETAILS = "INSERT INTO CONNECT_DETAILS (" + + "ACTION_ID, SOURCE_ID, SOURCE_NAME, SOURCE_TYPE, RELATIONSHIP, DESTINATION_ID, DESTINATION_NAME, DESTINATION_TYPE" + + ") VALUES (" + + "?, " + + "?, " + + "?, " + + "?, " + + "?, " + + "?, " + + "?, " + + "?" + + ")"; + + private static final String INSERT_MOVE_DETAILS = "INSERT INTO MOVE_DETAILS (" + + "ACTION_ID, GROUP_ID, GROUP_NAME, PREVIOUS_GROUP_ID, PREVIOUS_GROUP_NAME" + + ") VALUES (" + + "?, " + + "?, " + + "?, " + + "?, " + + "?" + + ")"; + + private static final String INSERT_PURGE_DETAILS = "INSERT INTO PURGE_DETAILS (" + + "ACTION_ID, END_DATE" + + ") VALUES (" + + "?, " + + "?" + + ")"; + + // ------------ + // action table + // ------------ + private static final String SELECT_ACTIONS = "SELECT * FROM ACTION"; + + private static final String SELECT_ACTION_COUNT = "SELECT COUNT(*) AS ACTION_COUNT FROM ACTION"; + + private static final String SELECT_ACTION_BY_ID = "SELECT * " + + "FROM ACTION " + + "WHERE " + + "ID = ?"; + + private static final String DELETE_ACTIONS = "DELETE FROM ACTION WHERE ACTION_TIMESTAMP < ?"; + + private static final String DELETE_SPECIFIC_ACTIONS = "DELETE FROM %s WHERE %s IN (SELECT ID FROM ACTION WHERE ACTION_TIMESTAMP < ?)"; + + // ----------------- + // component details + // ----------------- + private static final String SELECT_PROCESSOR_DETAILS_FOR_ACTION = "SELECT * FROM PROCESSOR_DETAILS WHERE ACTION_ID = ?"; + + private static final String SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION = "SELECT * FROM REMOTE_PROCESS_GROUP_DETAILS WHERE ACTION_ID = ?"; + + // -------------- + // action details + // -------------- + private static final String SELECT_MOVE_DETAILS_FOR_ACTION = "SELECT * FROM MOVE_DETAILS WHERE ACTION_ID = ?"; + + private static final String SELECT_CONFIGURE_DETAILS_FOR_ACTION = "SELECT * FROM CONFIGURE_DETAILS WHERE ACTION_ID = ?"; + + private static final String SELECT_CONNECT_DETAILS_FOR_ACTION = "SELECT * FROM CONNECT_DETAILS WHERE ACTION_ID = ?"; + + private static final String SELECT_PURGE_DETAILS_FOR_ACTION = "SELECT * FROM PURGE_DETAILS WHERE ACTION_ID = ?"; + + // --------------- + // previous values + // --------------- + private static final String SELECT_PREVIOUSLY_CONFIGURED_FIELDS = "SELECT DISTINCT CD.NAME " + + "FROM CONFIGURE_DETAILS CD " + + "INNER JOIN ACTION A " + + "ON CD.ACTION_ID = A.ID " + + "WHERE A.SOURCE_ID = ?"; + + private static final String SELECT_PREVIOUS_VALUES = "SELECT CD.VALUE, " + + "A.ACTION_TIMESTAMP, " + + "A.USER_NAME " + + "FROM CONFIGURE_DETAILS CD " + + "INNER JOIN ACTION A " + + "ON CD.ACTION_ID = A.ID " + + "WHERE A.SOURCE_ID = ? AND CD.NAME = ? " + + "ORDER BY A.ACTION_TIMESTAMP DESC " + + "LIMIT 4"; + + private Connection connection; + private Map columnMap; + + public StandardActionDAO(Connection connection) { + this.connection = connection; + + // initialize the column mappings + this.columnMap = new HashMap<>(); + this.columnMap.put("timestamp", "ACTION_TIMESTAMP"); + this.columnMap.put("sourceName", "SOURCE_NAME"); + this.columnMap.put("sourceType", "SOURCE_TYPE"); + this.columnMap.put("operation", "OPERATION"); + this.columnMap.put("userName", "USER_NAME"); + } + + @Override + public void createAction(Action action) throws DataAccessException { + if (action.getUserDn() == null) { + throw new IllegalArgumentException("User cannot be null."); + } + + if (action.getTimestamp() == null) { + throw new IllegalArgumentException("Action timestamp cannot be null."); + } + + PreparedStatement statement = null; + ResultSet rs = null; + try { + // obtain a statement to insert to the action table + statement = connection.prepareStatement(INSERT_ACTION, Statement.RETURN_GENERATED_KEYS); + statement.setString(1, StringUtils.left(action.getUserDn(), 255)); + statement.setString(2, StringUtils.left(action.getUserName(), 100)); + statement.setString(3, action.getSourceId()); + statement.setString(4, StringUtils.left(action.getSourceName(), 1000)); + statement.setString(5, action.getSourceType().toString()); + statement.setString(6, action.getOperation().toString()); + statement.setTimestamp(7, new java.sql.Timestamp(action.getTimestamp().getTime())); + + // insert the action + int updateCount = statement.executeUpdate(); + + // get the action id + rs = statement.getGeneratedKeys(); + if (updateCount == 1 && rs.next()) { + action.setId(rs.getInt(1)); + } else { + throw new DataAccessException("Unable to insert action."); + } + + // close the previous statement + statement.close(); + + // determine the type of component + ComponentDetails componentDetails = action.getComponentDetails(); + if (componentDetails instanceof ProcessorDetails) { + createProcessorDetails(action.getId(), (ProcessorDetails) componentDetails); + } else if (componentDetails instanceof RemoteProcessGroupDetails) { + createRemoteProcessGroupDetails(action.getId(), (RemoteProcessGroupDetails) componentDetails); + } + + // determine the type of action + ActionDetails details = action.getActionDetails(); + if (details instanceof ConnectDetails) { + createConnectDetails(action.getId(), (ConnectDetails) details); + } else if (details instanceof MoveDetails) { + createMoveDetails(action.getId(), (MoveDetails) details); + } else if (details instanceof ConfigureDetails) { + createConfigureDetails(action.getId(), (ConfigureDetails) details); + } else if (details instanceof PurgeDetails) { + createPurgeDetails(action.getId(), (PurgeDetails) details); + } + + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Persists the processor details. + * + * @param actionId + * @param processorDetails + * @throws DataAccessException + */ + private void createProcessorDetails(int actionId, ProcessorDetails processorDetails) throws DataAccessException { + PreparedStatement statement = null; + try { + // obtain a statement to insert to the processor action table + statement = connection.prepareStatement(INSERT_PROCESSOR_DETAILS); + statement.setInt(1, actionId); + statement.setString(2, StringUtils.left(processorDetails.getType(), 1000)); + + // insert the action + int updateCount = statement.executeUpdate(); + + // ensure the operation completed successfully + if (updateCount != 1) { + throw new DataAccessException("Unable to insert processor details."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Persists the remote process group details. + * + * @param actionId + * @param remoteProcessGroupDetails + * @throws DataAccessException + */ + private void createRemoteProcessGroupDetails(int actionId, RemoteProcessGroupDetails remoteProcessGroupDetails) throws DataAccessException { + PreparedStatement statement = null; + try { + // obtain a statement to insert to the processor action table + statement = connection.prepareStatement(INSERT_REMOTE_PROCESS_GROUP_DETAILS); + statement.setInt(1, actionId); + statement.setString(2, StringUtils.left(remoteProcessGroupDetails.getUri(), 2500)); + + // insert the action + int updateCount = statement.executeUpdate(); + + // ensure the operation completed successfully + if (updateCount != 1) { + throw new DataAccessException("Unable to insert remote prcoess group details."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Persists the connection details. + * + * @param actionId + * @param connectionDetails + * @throws DataAccessException + */ + private void createConnectDetails(int actionId, ConnectDetails connectionDetails) throws DataAccessException { + PreparedStatement statement = null; + try { + // obtain a statement to insert to the processor action table + statement = connection.prepareStatement(INSERT_CONNECT_DETAILS); + statement.setInt(1, actionId); + statement.setString(2, connectionDetails.getSourceId()); + statement.setString(3, StringUtils.left(connectionDetails.getSourceName(), 1000)); + statement.setString(4, StringUtils.left(connectionDetails.getSourceType().toString(), 1000)); + statement.setString(5, StringUtils.left(connectionDetails.getRelationship(), 1000)); + statement.setString(6, connectionDetails.getDestinationId()); + statement.setString(7, StringUtils.left(connectionDetails.getDestinationName(), 1000)); + statement.setString(8, StringUtils.left(connectionDetails.getDestinationType().toString(), 1000)); + + // insert the action + int updateCount = statement.executeUpdate(); + + // ensure the operation completed successfully + if (updateCount != 1) { + throw new DataAccessException("Unable to insert connection details."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Persists the move details. + * + * @param actionId + * @param moveDetails + * @throws DataAccessException + */ + private void createMoveDetails(int actionId, MoveDetails moveDetails) throws DataAccessException { + PreparedStatement statement = null; + try { + // obtain a statement to insert to the processor action table + statement = connection.prepareStatement(INSERT_MOVE_DETAILS); + statement.setInt(1, actionId); + statement.setString(2, moveDetails.getGroupId()); + statement.setString(3, StringUtils.left(moveDetails.getGroup(), 1000)); + statement.setString(4, moveDetails.getPreviousGroupId()); + statement.setString(5, StringUtils.left(moveDetails.getPreviousGroup(), 1000)); + + // insert the action + int updateCount = statement.executeUpdate(); + + // ensure the operation completed successfully + if (updateCount != 1) { + throw new DataAccessException("Unable to insert move details."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Persists the configuration details. + * + * @param actionId + * @param configurationDetails + * @throws DataAccessException + */ + private void createConfigureDetails(int actionId, ConfigureDetails configurationDetails) throws DataAccessException { + PreparedStatement statement = null; + try { + // obtain a statement to insert to the processor action table + statement = connection.prepareStatement(INSERT_CONFIGURE_DETAILS); + statement.setInt(1, actionId); + statement.setString(2, StringUtils.left(configurationDetails.getName(), 1000)); + statement.setString(3, StringUtils.left(configurationDetails.getValue(), 5000)); + statement.setString(4, StringUtils.left(configurationDetails.getPreviousValue(), 5000)); + + // insert the action + int updateCount = statement.executeUpdate(); + + // ensure the operation completed successfully + if (updateCount != 1) { + throw new DataAccessException("Unable to insert configure details."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Persists the purge details. + * + * @param actionId + * @param purgeDetails + * @throws DataAccessException + */ + private void createPurgeDetails(int actionId, PurgeDetails purgeDetails) throws DataAccessException { + PreparedStatement statement = null; + try { + // obtain a statement to insert to the processor action table + statement = connection.prepareStatement(INSERT_PURGE_DETAILS); + statement.setInt(1, actionId); + statement.setTimestamp(2, new java.sql.Timestamp(purgeDetails.getEndDate().getTime())); + + // insert the action + int updateCount = statement.executeUpdate(); + + // ensure the operation completed successfully + if (updateCount != 1) { + throw new DataAccessException("Unable to insert connection details."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + /** + * Finds actions that meet the criteria in the specified query. + * + * @param historyQuery + * @return + * @throws DataAccessException + */ + @Override + public History findActions(HistoryQuery historyQuery) throws DataAccessException { + + // get the sort column + String sortColumn = "ACTION_TIMESTAMP"; + if (StringUtils.isNotBlank(historyQuery.getSortColumn())) { + String rawColumnName = historyQuery.getSortColumn(); + if (!columnMap.containsKey(rawColumnName)) { + throw new IllegalArgumentException(String.format("Unrecognized column name '%s'.", rawColumnName)); + } + sortColumn = columnMap.get(rawColumnName); + } + + // get the sort order + String sortOrder = "desc"; + if (StringUtils.isNotBlank(historyQuery.getSortOrder())) { + sortOrder = historyQuery.getSortOrder(); + } + + History actionResult = new History(); + Collection actions = new ArrayList<>(); + PreparedStatement statement = null; + ResultSet rs = null; + try { + List where = new ArrayList<>(); + + // append the start time + if (historyQuery.getStartDate() != null) { + where.add("ACTION_TIMESTAMP >= ?"); + } + + // append the end time + if (historyQuery.getEndDate() != null) { + where.add("ACTION_TIMESTAMP <= ?"); + } + + // append the user id as necessary + if (historyQuery.getUserName() != null) { + where.add("UPPER(USER_NAME) LIKE ?"); + } + + // append the source id as necessary + if (historyQuery.getSourceId() != null) { + where.add("SOURCE_ID = ?"); + } + + String sql = SELECT_ACTION_COUNT; + if (!where.isEmpty()) { + sql += " WHERE " + StringUtils.join(where, " AND "); + } + + // get the total number of actions + statement = connection.prepareStatement(sql); + int paramIndex = 1; + + // set the start date as necessary + if (historyQuery.getStartDate() != null) { + statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getStartDate().getTime())); + } + + // set the end date as necessary + if (historyQuery.getEndDate() != null) { + statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getEndDate().getTime())); + } + + // set the user id as necessary + if (historyQuery.getUserName() != null) { + statement.setString(paramIndex++, "%" + historyQuery.getUserName().toUpperCase() + "%"); + } + + // set the source id as necessary + if (historyQuery.getSourceId() != null) { + statement.setString(paramIndex, historyQuery.getSourceId()); + } + + // execute the statement + rs = statement.executeQuery(); + + // ensure there are results + if (rs.next()) { + actionResult.setTotal(rs.getInt("ACTION_COUNT")); + } else { + throw new DataAccessException("Unable to determine total action count."); + } + + sql = SELECT_ACTIONS; + if (!where.isEmpty()) { + sql += " WHERE " + StringUtils.join(where, " AND "); + } + + // append the sort criteria + sql += (" ORDER BY " + sortColumn + " " + sortOrder); + + // append the offset and limit + sql += " LIMIT ? OFFSET ?"; + + // close the previous statement + statement.close(); + + // create the statement + statement = connection.prepareStatement(sql); + paramIndex = 1; + + // set the start date as necessary + if (historyQuery.getStartDate() != null) { + statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getStartDate().getTime())); + } + + // set the end date as necessary + if (historyQuery.getEndDate() != null) { + statement.setTimestamp(paramIndex++, new java.sql.Timestamp(historyQuery.getEndDate().getTime())); + } + + // set the user id as necessary + if (historyQuery.getUserName() != null) { + statement.setString(paramIndex++, "%" + historyQuery.getUserName().toUpperCase() + "%"); + } + + // set the source id as necessary + if (historyQuery.getSourceId() != null) { + statement.setString(paramIndex++, historyQuery.getSourceId()); + } + + // set the limit + statement.setInt(paramIndex++, historyQuery.getCount()); + + // set the offset according to the currented page calculated above + statement.setInt(paramIndex, historyQuery.getOffset()); + + // execute the query + rs = statement.executeQuery(); + + // create each corresponding action + while (rs.next()) { + final Integer actionId = rs.getInt("ID"); + final Operation operation = Operation.valueOf(rs.getString("OPERATION")); + final Component component = Component.valueOf(rs.getString("SOURCE_TYPE")); + + Action action = new Action(); + action.setId(actionId); + action.setUserDn(rs.getString("USER_DN")); + action.setUserName(rs.getString("USER_NAME")); + action.setOperation(Operation.valueOf(rs.getString("OPERATION"))); + action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime())); + action.setSourceId(rs.getString("SOURCE_ID")); + action.setSourceName(rs.getString("SOURCE_NAME")); + action.setSourceType(Component.valueOf(rs.getString("SOURCE_TYPE"))); + + // get the component details if appropriate + ComponentDetails componentDetails = null; + if (Component.Processor.equals(component)) { + componentDetails = getProcessorDetails(actionId); + } else if (Component.RemoteProcessGroup.equals(component)) { + componentDetails = getRemoteProcessGroupDetails(actionId); + } + + if (componentDetails != null) { + action.setComponentDetails(componentDetails); + } + + // get the action details if appropriate + ActionDetails actionDetails = null; + if (Operation.Move.equals(operation)) { + actionDetails = getMoveDetails(actionId); + } else if (Operation.Configure.equals(operation)) { + actionDetails = getConfigureDetails(actionId); + } else if (Operation.Connect.equals(operation) || Operation.Disconnect.equals(operation)) { + actionDetails = getConnectDetails(actionId); + } else if (Operation.Purge.equals(operation)) { + actionDetails = getPurgeDetails(actionId); + } + + // set the action details + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + + // add the action + actions.add(action); + } + + // populate the action result + actionResult.setActions(actions); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return actionResult; + } + + @Override + public Action getAction(Integer actionId) throws DataAccessException { + Action action = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_ACTION_BY_ID); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + Operation operation = Operation.valueOf(rs.getString("OPERATION")); + Component component = Component.valueOf(rs.getString("SOURCE_TYPE")); + + // populate the action + action = new Action(); + action.setId(rs.getInt("ID")); + action.setUserDn(rs.getString("USER_DN")); + action.setUserName(rs.getString("USER_NAME")); + action.setOperation(operation); + action.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime())); + action.setSourceId(rs.getString("SOURCE_ID")); + action.setSourceName(rs.getString("SOURCE_NAME")); + action.setSourceType(component); + + // get the component details if appropriate + ComponentDetails componentDetails = null; + if (Component.Processor.equals(component)) { + componentDetails = getProcessorDetails(actionId); + } else if (Component.RemoteProcessGroup.equals(component)) { + componentDetails = getRemoteProcessGroupDetails(actionId); + } + + if (componentDetails != null) { + action.setComponentDetails(componentDetails); + } + + // get the action details if appropriate + ActionDetails actionDetails = null; + if (Operation.Move.equals(operation)) { + actionDetails = getMoveDetails(actionId); + } else if (Operation.Configure.equals(operation)) { + actionDetails = getConfigureDetails(actionId); + } else if (Operation.Connect.equals(operation) || Operation.Disconnect.equals(operation)) { + actionDetails = getConnectDetails(actionId); + } else if (Operation.Purge.equals(operation)) { + actionDetails = getPurgeDetails(actionId); + } + + // set the action details + if (actionDetails != null) { + action.setActionDetails(actionDetails); + } + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return action; + } + + /** + * Loads the specified processor details. + * + * @param actionId + * @return + * @throws DataAccessException + */ + private ProcessorDetails getProcessorDetails(Integer actionId) throws DataAccessException { + ProcessorDetails processorDetails = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_PROCESSOR_DETAILS_FOR_ACTION); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + processorDetails = new ProcessorDetails(); + processorDetails.setType(rs.getString("TYPE")); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return processorDetails; + } + + /** + * Loads the specified remote process group details. + * + * @param actionId + * @return + * @throws DataAccessException + */ + private RemoteProcessGroupDetails getRemoteProcessGroupDetails(Integer actionId) throws DataAccessException { + RemoteProcessGroupDetails remoteProcessGroupDetails = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_REMOTE_PROCESS_GROUP_DETAILS_FOR_ACTION); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + remoteProcessGroupDetails = new RemoteProcessGroupDetails(); + remoteProcessGroupDetails.setUri(rs.getString("URI")); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return remoteProcessGroupDetails; + } + + /** + * Loads the specified move details. + * + * @param actionId + * @return + * @throws DataAccessException + */ + private MoveDetails getMoveDetails(Integer actionId) throws DataAccessException { + MoveDetails moveDetails = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_MOVE_DETAILS_FOR_ACTION); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + moveDetails = new MoveDetails(); + moveDetails.setGroupId(rs.getString("GROUP_ID")); + moveDetails.setGroup(rs.getString("GROUP_NAME")); + moveDetails.setPreviousGroupId(rs.getString("PREVIOUS_GROUP_ID")); + moveDetails.setPreviousGroup(rs.getString("PREVIOUS_GROUP_NAME")); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return moveDetails; + } + + /** + * Loads the specified relationship details. + * + * @param actionId + * @return + * @throws DataAccessException + */ + private ConnectDetails getConnectDetails(Integer actionId) throws DataAccessException { + ConnectDetails connectionDetails = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_CONNECT_DETAILS_FOR_ACTION); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + final Component sourceComponent = Component.valueOf(rs.getString("SOURCE_TYPE")); + final Component destinationComponent = Component.valueOf(rs.getString("DESTINATION_TYPE")); + + connectionDetails = new ConnectDetails(); + connectionDetails.setSourceId(rs.getString("SOURCE_ID")); + connectionDetails.setSourceName(rs.getString("SOURCE_NAME")); + connectionDetails.setSourceType(sourceComponent); + connectionDetails.setRelationship(rs.getString("RELATIONSHIP")); + connectionDetails.setDestinationId(rs.getString("DESTINATION_ID")); + connectionDetails.setDestinationName(rs.getString("DESTINATION_NAME")); + connectionDetails.setDestinationType(destinationComponent); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return connectionDetails; + } + + /** + * Loads the specified configuration details. + * + * @param actionId + * @return + * @throws DataAccessException + */ + private ConfigureDetails getConfigureDetails(Integer actionId) throws DataAccessException { + ConfigureDetails configurationDetails = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_CONFIGURE_DETAILS_FOR_ACTION); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + configurationDetails = new ConfigureDetails(); + configurationDetails.setName(rs.getString("NAME")); + configurationDetails.setValue(rs.getString("VALUE")); + configurationDetails.setPreviousValue(rs.getString("PREVIOUS_VALUE")); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return configurationDetails; + } + + /** + * Loads the specified purge details. + * + * @param actionId + * @return + * @throws DataAccessException + */ + private PurgeDetails getPurgeDetails(Integer actionId) throws DataAccessException { + PurgeDetails purgeDetails = null; + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_PURGE_DETAILS_FOR_ACTION); + statement.setInt(1, actionId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + if (rs.next()) { + purgeDetails = new PurgeDetails(); + purgeDetails.setEndDate(new Date(rs.getTimestamp("END_DATE").getTime())); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return purgeDetails; + } + + @Override + public Map> getPreviousValues(String processorId) { + Map> previousValues = new LinkedHashMap<>(); + + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_PREVIOUSLY_CONFIGURED_FIELDS); + statement.setString(1, processorId); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + while (rs.next()) { + final String property = rs.getString("NAME"); + previousValues.put(property, getPreviousValuesForProperty(processorId, property)); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return previousValues; + } + + private List getPreviousValuesForProperty(final String processorId, final String property) { + List previousValues = new ArrayList<>(); + + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the statement + statement = connection.prepareStatement(SELECT_PREVIOUS_VALUES); + statement.setString(1, processorId); + statement.setString(2, property); + + // execute the query + rs = statement.executeQuery(); + + // ensure results + while (rs.next()) { + // get the previous value + final PreviousValue previousValue = new PreviousValue(); + previousValue.setPreviousValue(rs.getString("VALUE")); + previousValue.setTimestamp(new Date(rs.getTimestamp("ACTION_TIMESTAMP").getTime())); + previousValue.setUserName(rs.getString("USER_NAME")); + previousValues.add(previousValue); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return previousValues; + } + + @Override + public void deleteActions(Date endDate) throws DataAccessException { + PreparedStatement statement = null; + try { + // ----------------- + // component details + // ----------------- + + // create the move delete statement + statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "PROCESSOR_DETAILS", "ACTION_ID")); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + statement.close(); + + // create the move delete statement + statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "REMOTE_PROCESS_GROUP_DETAILS", "ACTION_ID")); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + statement.close(); + + // -------------- + // action details + // -------------- + // create the move delete statement + statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "MOVE_DETAILS", "ACTION_ID")); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + statement.close(); + + // create the configure delete statement + statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "CONFIGURE_DETAILS", "ACTION_ID")); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + statement.close(); + + // create the connect delete statement + statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "CONNECT_DETAILS", "ACTION_ID")); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + statement.close(); + + // create the relationship delete statement + statement = connection.prepareStatement(String.format(DELETE_SPECIFIC_ACTIONS, "PURGE_DETAILS", "ACTION_ID")); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + statement.close(); + + // ------- + // actions + // ------- + // create the action delete statement + statement = connection.prepareStatement(DELETE_ACTIONS); + statement.setTimestamp(1, new java.sql.Timestamp(endDate.getTime())); + statement.executeUpdate(); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java new file mode 100644 index 0000000000..4e2cc26db5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardAuthorityDAO.java @@ -0,0 +1,172 @@ +/* + * 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.admin.dao.impl; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.EnumSet; +import java.util.Set; +import org.apache.nifi.admin.RepositoryUtils; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.authorization.Authority; + +/** + * + */ +public class StandardAuthorityDAO implements AuthorityDAO { + + private static final String SELECT_AUTHORITIES_FOR_USER = "SELECT ID, ROLE " + + "FROM AUTHORITY " + + "WHERE USER_ID = ?"; + + private static final String INSERT_AUTHORITY = "INSERT INTO AUTHORITY (" + + "USER_ID, ROLE" + + ") VALUES (" + + "?, ?" + + ")"; + + private static final String DELETE_AUTHORITY = "DELETE FROM AUTHORITY " + + "WHERE USER_ID = ? AND ROLE = ?"; + + private static final String DELETE_AUTHORITIES_FOR_USER = "DELETE FROM AUTHORITY " + + "WHERE USER_ID = ?"; + + private final Connection connection; + + public StandardAuthorityDAO(Connection connection) { + this.connection = connection; + } + + @Override + public void createAuthorities(Set authorities, String userId) throws DataAccessException { + if (authorities == null) { + throw new IllegalArgumentException("Specified authorities cannot be null."); + } + + // ensure there are some authorities to create + if (!authorities.isEmpty()) { + PreparedStatement statement = null; + try { + // add each authority for the specified user + statement = connection.prepareStatement(INSERT_AUTHORITY); + statement.setString(1, userId); + for (Authority authority : authorities) { + statement.setString(2, authority.toString()); + statement.addBatch(); + } + + // insert the authorities + int[] updateCounts = statement.executeBatch(); + for (int updateCount : updateCounts) { + if (updateCount != 1) { + throw new DataAccessException("Unable to insert user authorities."); + } + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + } + + @Override + public void deleteAuthorities(String userId) throws DataAccessException { + // ensure there are some authorities to create + PreparedStatement statement = null; + try { + // add each authority for the specified user + statement = connection.prepareStatement(DELETE_AUTHORITIES_FOR_USER); + statement.setString(1, userId); + + // insert the authorities + statement.executeUpdate(); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void deleteAuthorities(Set authorities, String userId) throws DataAccessException { + if (authorities == null) { + throw new IllegalArgumentException("Specified authorities cannot be null."); + } + + // ensure there are some authorities to create + if (!authorities.isEmpty()) { + PreparedStatement statement = null; + try { + // add each authority for the specified user + statement = connection.prepareStatement(DELETE_AUTHORITY); + statement.setString(1, userId); + for (Authority authority : authorities) { + statement.setString(2, authority.toString()); + statement.addBatch(); + } + + // insert the authorities + int[] updateCounts = statement.executeBatch(); + for (int updateCount : updateCounts) { + if (updateCount != 1) { + throw new DataAccessException("Unable to remove user authorities."); + } + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + } + + @Override + public Set findAuthoritiesByUserId(String userId) throws DataAccessException { + Set authorities = EnumSet.noneOf(Authority.class); + PreparedStatement statement = null; + ResultSet rs = null; + try { + // add each authority for the specified user + statement = connection.prepareStatement(SELECT_AUTHORITIES_FOR_USER); + statement.setString(1, userId); + + // execute the query + rs = statement.executeQuery(); + + // create each corresponding authority + while (rs.next()) { + authorities.add(Authority.valueOfAuthority(rs.getString("ROLE"))); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + + return authorities; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java new file mode 100644 index 0000000000..ea7c1a188f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/dao/impl/StandardUserDAO.java @@ -0,0 +1,634 @@ +/* + * 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.admin.dao.impl; + +import java.nio.charset.StandardCharsets; +import java.sql.Connection; +import org.apache.nifi.admin.dao.UserDAO; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Types; +import java.util.Date; +import java.util.HashSet; +import java.util.Set; +import java.util.UUID; +import org.apache.nifi.admin.RepositoryUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; + +/** + * Responsible for loading and persisting NiFiUsers. + */ +public class StandardUserDAO implements UserDAO { + + private static final String SELECT_PENDING_ACCOUNTS_COUNT = "SELECT " + + "COUNT(*) as PENDING_ACCOUNTS " + + "FROM USER U " + + "WHERE U.STATUS = 'PENDING'"; + + private static final String SELECT_USER_BY_DN = "SELECT " + + "U.ID, " + + "U.DN, " + + "U.USER_NAME, " + + "U.USER_GROUP, " + + "U.CREATION, " + + "U.LAST_ACCESSED, " + + "U.LAST_VERIFIED, " + + "U.JUSTIFICATION, " + + "U.STATUS, " + + "A.ROLE " + + "FROM USER U " + + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched + + "ON U.ID = A.USER_ID " + + "WHERE U.DN = ?"; + + private static final String SELECT_USER_BY_ID = "SELECT " + + "U.ID, " + + "U.DN, " + + "U.USER_NAME, " + + "U.USER_GROUP, " + + "U.CREATION, " + + "U.LAST_ACCESSED, " + + "U.LAST_VERIFIED, " + + "U.JUSTIFICATION, " + + "U.STATUS, " + + "A.ROLE " + + "FROM USER U " + + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched + + "ON U.ID = A.USER_ID " + + "WHERE U.ID = ?"; + + private static final String SELECT_USERS = "SELECT " + + "U.ID, " + + "U.DN, " + + "U.USER_NAME, " + + "U.USER_GROUP, " + + "U.CREATION, " + + "U.LAST_ACCESSED, " + + "U.LAST_VERIFIED, " + + "U.JUSTIFICATION, " + + "U.STATUS, " + + "A.ROLE " + + "FROM USER U " + + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched + + "ON U.ID = A.USER_ID " + + "WHERE U.DN <> ?"; + + private static final String SELECT_USER_GROUPS = "SELECT DISTINCT " + + "U.USER_GROUP " + + "FROM USER U"; + + private static final String SELECT_USER_GROUP = "SELECT " + + "U.ID, " + + "U.DN, " + + "U.USER_NAME, " + + "U.USER_GROUP, " + + "U.CREATION, " + + "U.LAST_ACCESSED, " + + "U.LAST_VERIFIED, " + + "U.JUSTIFICATION, " + + "U.STATUS, " + + "A.ROLE " + + "FROM USER U " + + "LEFT JOIN AUTHORITY A " // ensures that users without authorities are still matched + + "ON U.ID = A.USER_ID " + + "WHERE U.DN <> ? AND U.USER_GROUP = ?"; + + private static final String INSERT_USER = "INSERT INTO USER (" + + "ID, DN, USER_NAME, USER_GROUP, CREATION, LAST_VERIFIED, JUSTIFICATION, STATUS" + + ") VALUES (" + + "?, " + + "?, " + + "?, " + + "?, " + + "NOW(), " + + "?, " + + "?, " + + "?" + + ")"; + + private static final String UPDATE_USER = "UPDATE USER SET " + + "DN = ?, " + + "USER_NAME = ?, " + + "USER_GROUP = ?, " + + "LAST_ACCESSED = ?, " + + "LAST_VERIFIED = ?, " + + "JUSTIFICATION = ?, " + + "STATUS = ? " + + "WHERE ID = ?"; + + private static final String UPDATE_USER_GROUP_STATUS = "UPDATE USER SET " + + "STATUS = ?," + + "USER_GROUP = NULL " + + "WHERE USER_GROUP = ?"; + + private static final String UPDATE_USER_GROUP_VERIFICATION = "UPDATE USER SET " + + "LAST_VERIFIED = ? " + + "WHERE USER_GROUP = ?"; + + private static final String UNGROUP_GROUP = "UPDATE USER SET " + + "USER_GROUP = NULL " + + "WHERE USER_GROUP = ?"; + + private static final String DELETE_USER = "DELETE FROM USER " + + "WHERE ID = ?"; + + private final Connection connection; + + public StandardUserDAO(Connection connection) { + this.connection = connection; + } + + @Override + public Boolean hasPendingUserAccounts() throws DataAccessException { + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the connection and obtain a statement + statement = connection.prepareStatement(SELECT_PENDING_ACCOUNTS_COUNT); + + // execute the query + rs = statement.executeQuery(); + + // get the first row which will contain the number of pending accounts + if (rs.next()) { + int pendingAccounts = rs.getInt("PENDING_ACCOUNTS"); + return pendingAccounts > 0; + } + + // query returned no results? + return false; + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public Set findUsers() throws DataAccessException { + Set users = new HashSet<>(); + + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the connection and obtain a statement + statement = connection.prepareStatement(SELECT_USERS); + statement.setString(1, NiFiUser.ANONYMOUS_USER_DN); + + // execute the query + rs = statement.executeQuery(); + + // create the user + NiFiUser user = null; + + // go through the user and its roles + while (rs.next()) { + // get the user id for the current record + String userId = rs.getString("ID"); + + // create the user during the first iteration + if (user == null || !userId.equals(user.getId())) { + user = new NiFiUser(); + user.setId(userId); + user.setDn(rs.getString("DN")); + user.setUserName(rs.getString("USER_NAME")); + user.setUserGroup(rs.getString("USER_GROUP")); + user.setJustification(rs.getString("JUSTIFICATION")); + user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS"))); + + // set the creation date + user.setCreation(new Date(rs.getTimestamp("CREATION").getTime())); + + // get the last accessed date + if (rs.getTimestamp("LAST_ACCESSED") != null) { + user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime())); + } + + // get the last verified date + if (rs.getTimestamp("LAST_VERIFIED") != null) { + user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime())); + } + + // add the user + users.add(user); + } + + // the select statement performs a left join since the desired + // user may not have any authorities + String authority = rs.getString("ROLE"); + if (StringUtils.isNotBlank(authority)) { + user.getAuthorities().add(Authority.valueOfAuthority(authority)); + } + } + + return users; + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public Set findUserGroups() throws DataAccessException { + Set userGroups = new HashSet<>(); + + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the connection and obtain a statement + statement = connection.prepareStatement(SELECT_USER_GROUPS); + + // execute the query + rs = statement.executeQuery(); + + // get each user group + while (rs.next()) { + userGroups.add(rs.getString("USER_GROUP")); + } + + return userGroups; + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public Set findUsersForGroup(String group) throws DataAccessException { + Set users = new HashSet<>(); + + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the connection and obtain a statement + statement = connection.prepareStatement(SELECT_USER_GROUP); + statement.setString(1, NiFiUser.ANONYMOUS_USER_DN); + statement.setString(2, group); + + // execute the query + rs = statement.executeQuery(); + + // create the user + NiFiUser user = null; + + // go through the user and its roles + while (rs.next()) { + // get the user id for the current record + String userId = rs.getString("ID"); + + // create the user during the first iteration + if (user == null || !userId.equals(user.getId())) { + user = new NiFiUser(); + user.setId(userId); + user.setDn(rs.getString("DN")); + user.setUserName(rs.getString("USER_NAME")); + user.setUserGroup(rs.getString("USER_GROUP")); + user.setJustification(rs.getString("JUSTIFICATION")); + user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS"))); + + // set the creation date + user.setCreation(new Date(rs.getTimestamp("CREATION").getTime())); + + // get the last accessed date + if (rs.getTimestamp("LAST_ACCESSED") != null) { + user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime())); + } + + // get the last verified date + if (rs.getTimestamp("LAST_VERIFIED") != null) { + user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime())); + } + + // add the user + users.add(user); + } + + // the select statement performs a left join since the desired + // user may not have any authorities + String authority = rs.getString("ROLE"); + if (StringUtils.isNotBlank(authority)) { + user.getAuthorities().add(Authority.valueOfAuthority(authority)); + } + } + + return users; + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public NiFiUser findUserById(String id) throws DataAccessException { + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the connection and obtain a statement + statement = connection.prepareStatement(SELECT_USER_BY_ID); + statement.setString(1, id); + + // execute the query + rs = statement.executeQuery(); + + // create the user + NiFiUser user = null; + + // go through the user and its roles + while (rs.next()) { + // create the user during the first iteration + if (user == null) { + user = new NiFiUser(); + user.setId(rs.getString("ID")); + user.setDn(rs.getString("DN")); + user.setUserName(rs.getString("USER_NAME")); + user.setUserGroup(rs.getString("USER_GROUP")); + user.setJustification(rs.getString("JUSTIFICATION")); + user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS"))); + + // set the creation date + user.setCreation(new Date(rs.getTimestamp("CREATION").getTime())); + + // get the last accessed date + if (rs.getTimestamp("LAST_ACCESSED") != null) { + user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime())); + } + + // get the last verified date + if (rs.getTimestamp("LAST_VERIFIED") != null) { + user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime())); + } + } + + // the select statement performs a left join since the desired + // user may not have any authorities + String authority = rs.getString("ROLE"); + if (StringUtils.isNotBlank(authority)) { + user.getAuthorities().add(Authority.valueOfAuthority(authority)); + } + } + + return user; + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public NiFiUser findUserByDn(String dn) throws DataAccessException { + PreparedStatement statement = null; + ResultSet rs = null; + try { + // create the connection and obtain a statement + statement = connection.prepareStatement(SELECT_USER_BY_DN); + statement.setString(1, dn); + + // execute the query + rs = statement.executeQuery(); + + // create the user + NiFiUser user = null; + + // go through the user and its roles + while (rs.next()) { + // create the user during the first iteration + if (user == null) { + user = new NiFiUser(); + user.setId(rs.getString("ID")); + user.setDn(rs.getString("DN")); + user.setUserName(rs.getString("USER_NAME")); + user.setUserGroup(rs.getString("USER_GROUP")); + user.setJustification(rs.getString("JUSTIFICATION")); + user.setStatus(AccountStatus.valueOfStatus(rs.getString("STATUS"))); + + // set the creation date + user.setCreation(new Date(rs.getTimestamp("CREATION").getTime())); + + // get the last accessed date + if (rs.getTimestamp("LAST_ACCESSED") != null) { + user.setLastAccessed(new Date(rs.getTimestamp("LAST_ACCESSED").getTime())); + } + + // get the last verified date + if (rs.getTimestamp("LAST_VERIFIED") != null) { + user.setLastVerified(new Date(rs.getTimestamp("LAST_VERIFIED").getTime())); + } + } + + // the select statement performs a left join since the desired + // user may not have any authorities + String authority = rs.getString("ROLE"); + if (StringUtils.isNotBlank(authority)) { + user.getAuthorities().add(Authority.valueOfAuthority(authority)); + } + } + + return user; + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void createUser(NiFiUser user) throws DataAccessException { + if (user.getDn() == null) { + throw new IllegalArgumentException("User dn must be specified."); + } + + PreparedStatement statement = null; + ResultSet rs = null; + try { + final String id = UUID.nameUUIDFromBytes(user.getDn().getBytes(StandardCharsets.UTF_8)).toString(); + + // create a statement + statement = connection.prepareStatement(INSERT_USER, Statement.RETURN_GENERATED_KEYS); + statement.setString(1, id); + statement.setString(2, StringUtils.left(user.getDn(), 255)); + statement.setString(3, StringUtils.left(user.getUserName(), 100)); + statement.setString(4, StringUtils.left(user.getUserGroup(), 100)); + if (user.getLastVerified() != null) { + statement.setTimestamp(5, new java.sql.Timestamp(user.getLastVerified().getTime())); + } else { + statement.setTimestamp(5, null); + } + statement.setString(6, StringUtils.left(user.getJustification(), 500)); + statement.setString(7, user.getStatus().toString()); + + // insert the user + int updateCount = statement.executeUpdate(); + if (updateCount == 1) { + user.setId(id); + } else { + throw new DataAccessException("Unable to insert user."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(rs); + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void deleteUser(String id) throws DataAccessException { + // ensure there are some authorities to create + PreparedStatement statement = null; + try { + // add each authority for the specified user + statement = connection.prepareStatement(DELETE_USER); + statement.setString(1, id); + + // insert the authorities + statement.executeUpdate(); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void updateUser(NiFiUser user) throws DataAccessException { + PreparedStatement statement = null; + try { + // create a statement + statement = connection.prepareStatement(UPDATE_USER); + statement.setString(1, StringUtils.left(user.getDn(), 255)); + statement.setString(2, StringUtils.left(user.getUserName(), 100)); + statement.setString(3, StringUtils.left(user.getUserGroup(), 100)); + statement.setString(6, StringUtils.left(user.getJustification(), 500)); + statement.setString(7, user.getStatus().toString()); + statement.setString(8, user.getId()); + + // set the last accessed time accordingly + if (user.getLastAccessed() == null) { + statement.setNull(4, Types.TIMESTAMP); + } else { + statement.setTimestamp(4, new java.sql.Timestamp(user.getLastAccessed().getTime())); + } + + // set the last verified time accordingly + if (user.getLastVerified() == null) { + statement.setNull(5, Types.TIMESTAMP); + } else { + statement.setTimestamp(5, new java.sql.Timestamp(user.getLastVerified().getTime())); + } + + // perform the update + int updateCount = statement.executeUpdate(); + if (updateCount != 1) { + throw new DataAccessException("Unable to update user."); + } + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void updateGroupStatus(String group, AccountStatus status) throws DataAccessException { + PreparedStatement statement = null; + try { + // create a statement + statement = connection.prepareStatement(UPDATE_USER_GROUP_STATUS); + statement.setString(1, status.toString()); + statement.setString(2, group); + + // perform the update + statement.executeUpdate(); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void updateGroupVerification(String group, Date lastVerified) throws DataAccessException { + PreparedStatement statement = null; + try { + // create a statement + statement = connection.prepareStatement(UPDATE_USER_GROUP_VERIFICATION); + + // set the last verified time accordingly + if (lastVerified == null) { + statement.setNull(1, Types.TIMESTAMP); + } else { + statement.setTimestamp(1, new java.sql.Timestamp(lastVerified.getTime())); + } + + // set the group + statement.setString(2, group); + + // perform the update + statement.executeUpdate(); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + + @Override + public void ungroup(String group) throws DataAccessException { + PreparedStatement statement = null; + try { + // create a statement + statement = connection.prepareStatement(UNGROUP_GROUP); + statement.setString(1, group); + + // perform the update + statement.executeUpdate(); + } catch (SQLException sqle) { + throw new DataAccessException(sqle); + } catch (DataAccessException dae) { + throw dae; + } finally { + RepositoryUtils.closeQuietly(statement); + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java new file mode 100644 index 0000000000..e8b3d1098e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountDisabledException.java @@ -0,0 +1,40 @@ +/* + * 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.admin.service; + +/** + * Exception to indicate that the user account is disabled. + */ +public class AccountDisabledException extends RuntimeException { + + public AccountDisabledException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public AccountDisabledException(Throwable cause) { + super(cause); + } + + public AccountDisabledException(String message, Throwable cause) { + super(message, cause); + } + + public AccountDisabledException(String message) { + super(message); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java new file mode 100644 index 0000000000..88287ce23b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountNotFoundException.java @@ -0,0 +1,40 @@ +/* + * 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.admin.service; + +/** + * Exception to indicate that the user account is disabled. + */ +public class AccountNotFoundException extends RuntimeException { + + public AccountNotFoundException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public AccountNotFoundException(Throwable cause) { + super(cause); + } + + public AccountNotFoundException(String message, Throwable cause) { + super(message, cause); + } + + public AccountNotFoundException(String message) { + super(message); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java new file mode 100644 index 0000000000..dacc483148 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AccountPendingException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service; + +/** + * Exception to indicate that the user has already submitting an account request + * and that request is still pending. + */ +public class AccountPendingException extends RuntimeException { + + public AccountPendingException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public AccountPendingException(Throwable cause) { + super(cause); + } + + public AccountPendingException(String message, Throwable cause) { + super(message, cause); + } + + public AccountPendingException(String message) { + super(message); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java new file mode 100644 index 0000000000..c0e8ac1ce9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AdministrationException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service; + +/** + * + */ +public class AdministrationException extends RuntimeException { + + public AdministrationException(Throwable cause) { + super(cause); + } + + public AdministrationException(String message, Throwable cause) { + super(message, cause); + } + + public AdministrationException(String message) { + super(message); + } + + public AdministrationException() { + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java new file mode 100644 index 0000000000..0843bd804a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/AuditService.java @@ -0,0 +1,76 @@ +/* + * 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.admin.service; + +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; +import org.apache.nifi.action.Action; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.History; +import org.apache.nifi.history.PreviousValue; + +/** + * Allows NiFi actions to be audited. + */ +public interface AuditService { + + /** + * Adds the specified actions. + * + * @param actions + * @throws AdministrationException + */ + void addActions(Collection actions); + + /** + * Finds the previous values for the specified property in the specified + * processor. Returns null if there are none. + * + * @param processorId + * @return + */ + Map> getPreviousValues(String processorId); + + /** + * Get the actions within the given date range. + * + * @param actionQuery + * @return + * @throws AdministrationException + */ + History getActions(HistoryQuery actionQuery); + + /** + * Get the details for the specified action id. If the action cannot be + * found, null is returned. + * + * @param actionId + * @return + */ + Action getAction(Integer actionId); + + /** + * Purges all action's that occurred before the specified end date. + * + * @param end + * @param purgeAction + * @throws AdministrationException + */ + void purgeActions(Date end, Action purgeAction); +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java new file mode 100644 index 0000000000..76e54d6ee1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/UserService.java @@ -0,0 +1,148 @@ +/* + * 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.admin.service; + +import java.util.Collection; +import java.util.Set; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.user.NiFiUserGroup; + +/** + * Manages NiFi user accounts. + */ +public interface UserService { + + /** + * Creates a new user account using the specified dn and justification. + * + * @param dn + * @param justification + * @return + */ + NiFiUser createPendingUserAccount(String dn, String justification); + + /** + * Determines if there are any PENDING user accounts present. + * + * @return + */ + Boolean hasPendingUserAccount(); + + /** + * Updates a user group using the specified group comprised of the specified + * users. Returns all the users that are currently in the specified group. + * + * @param group + * @param userIds + * @param authorities + * @return + */ + NiFiUserGroup updateGroup(String group, Set userIds, Set authorities); + + /** + * Authorizes the user specified. + * + * @param dn + * @return + */ + NiFiUser checkAuthorization(String dn); + + /** + * Deletes the user with the specified id. + * + * @param id + */ + void deleteUser(String id); + + /** + * Disables the specified users account. + * + * @param id + * @return + */ + NiFiUser disable(String id); + + /** + * Disables the specified user group. + * + * @param group + * @return + */ + NiFiUserGroup disableGroup(String group); + + /** + * Updates the specified user with the specified authorities. + * + * @param id + * @param authorities + * @return + */ + NiFiUser update(String id, Set authorities); + + /** + * Invalidates the specified user account. + * + * @param id + */ + void invalidateUserAccount(String id); + + /** + * Invalidates the user accounts associated with the specified user group. + * + * @param group + */ + void invalidateUserGroupAccount(String group); + + /** + * Ungroups the specified group. + * + * @param group + */ + void ungroup(String group); + + /** + * Ungroups the specified user. + * + * @param id + */ + void ungroupUser(String id); + + /** + * Returns a collection of all NiFiUsers. + * + * @return + */ + Collection getUsers(); + + /** + * Finds the specified user by id. + * + * @param id + * @return + */ + NiFiUser getUserById(String id); + + /** + * Finds the specified user by dn. + * + * @param dn + * @return + * @throws AdministrationException + */ + NiFiUser getUserByDn(String dn); +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java new file mode 100644 index 0000000000..41c97fe0cb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AbstractUserAction.java @@ -0,0 +1,97 @@ +/* + * 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.admin.service.action; + +import java.util.Date; +import java.util.EnumSet; +import java.util.Set; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; + +/** + * + * @param + */ +public abstract class AbstractUserAction implements AdministrationAction { + + /** + * Determines the authorities that need to be added to the specified user. + * + * @param user + * @param authorities + * @return + */ + protected Set determineAuthoritiesToAdd(NiFiUser user, Set authorities) { + // not using copyOf since authorities may be empty and copyOf can throw an IllegalArgumentException when empty + Set authoritiesToAdd = EnumSet.noneOf(Authority.class); + authoritiesToAdd.addAll(authorities); + + // identify the authorities that need to be inserted + authoritiesToAdd.removeAll(user.getAuthorities()); + + // return the desired authorities + return authoritiesToAdd; + } + + /** + * Determines the authorities that need to be removed from the specified + * user. + * + * @param user + * @param authorities + * @return + */ + protected Set determineAuthoritiesToRemove(NiFiUser user, Set authorities) { + Set authoritiesToRemove = EnumSet.copyOf(user.getAuthorities()); + + // identify the authorities that need to be removed + authoritiesToRemove.removeAll(authorities); + + // return the desired authorities + return authoritiesToRemove; + } + + /** + * Verifies the specified users account. Includes obtaining the authorities + * and group according to the specified authority provider. + * + * @param authorityProvider + * @param user + */ + protected void verifyAccount(AuthorityProvider authorityProvider, NiFiUser user) { + // load the roles for the user + Set authorities = authorityProvider.getAuthorities(user.getDn()); + + // update the user's authorities + user.getAuthorities().clear(); + user.getAuthorities().addAll(authorities); + + // get the user group + user.setUserGroup(authorityProvider.getGroupForUser(user.getDn())); + + // update the users status in case they were previously pending or disabled + user.setStatus(AccountStatus.ACTIVE); + + // update the users last verified time - this timestampt shouldn't be record + // until the both the user's authorities and group have been synced + Date now = new Date(); + user.setLastVerified(now); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java new file mode 100644 index 0000000000..5a2159fdfc --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AddActionsAction.java @@ -0,0 +1,48 @@ +/* + * 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.admin.service.action; + +import java.util.Collection; +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * Adds the specified actions. + */ +public class AddActionsAction implements AdministrationAction { + + private final Collection actions; + + public AddActionsAction(Collection actions) { + this.actions = actions; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + ActionDAO actionDao = daoFactory.getActionDAO(); + + // add each action + for (Action action : actions) { + actionDao.createAction(action); + } + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java new file mode 100644 index 0000000000..5818ebe073 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AdministrationAction.java @@ -0,0 +1,38 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * Defines the administration action. Actions are provided a DAO factory and + * authority provider to perform a require action. + * + * @param + */ +public interface AdministrationAction { + + /** + * Performs an action using the specified DAOFactory and AuthorityProvider. + * + * @param daoFactory + * @param authorityProvider + * @return + */ + T execute(DAOFactory daoFactory, AuthorityProvider authorityProvider); +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java new file mode 100644 index 0000000000..ea6973dd20 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/AuthorizeUserAction.java @@ -0,0 +1,175 @@ +/* + * 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.admin.service.action; + +import java.util.Calendar; +import java.util.Date; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountDisabledException; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AccountPendingException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.security.util.CertificateUtils; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class AuthorizeUserAction extends AbstractUserAction { + + private final String dn; + private final int cacheDurationSeconds; + + public AuthorizeUserAction(String dn, int cacheDurationSeconds) { + this.dn = dn; + this.cacheDurationSeconds = cacheDurationSeconds; + } + + @Override + public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + + // get the user + NiFiUser user = userDao.findUserByDn(dn); + + // verify the user was found + if (user == null) { + // determine whether this users exists + boolean doesDnExist = false; + try { + doesDnExist = authorityProvider.doesDnExist(dn); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae); + } + + // if the authority provider has the details for this user, create the account + if (doesDnExist) { + // create the user + user = new NiFiUser(); + user.setDn(dn); + user.setUserName(CertificateUtils.extractUsername(dn)); + user.setJustification("User details specified by authority provider."); + + try { + // verify the users account + verifyAccount(authorityProvider, user); + + // get the date used for verification + Date now = user.getLastVerified(); + + // update the last accessed field + user.setLastAccessed(now); + user.setCreation(now); + + // create the new user account + CreateUserAction createUser = new CreateUserAction(user); + createUser.execute(daoFactory, authorityProvider); + } catch (UnknownIdentityException uie) { + // strange since the provider just reported this dn existed but handleing anyways... + throw new AccountNotFoundException(String.format("Unable to verify access for %s.", dn)); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae); + } + } else { + throw new AccountNotFoundException(String.format("Unable to verify access for %s.", dn)); + } + } else { + Throwable providerError = null; + + // verify the users account if necessary + if (isAccountVerificationRequired(user)) { + try { + // verify the users account + verifyAccount(authorityProvider, user); + + // update the last accessed field + user.setLastAccessed(user.getLastVerified()); + } catch (UnknownIdentityException uie) { + // check the account status before attempting to update the account - depending on the account + // status we might not need to update the account + checkAccountStatus(user); + + // the user is currently active and they were not found in the providers - disable the account... + user.setStatus(AccountStatus.DISABLED); + + // record the exception + providerError = uie; + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae); + } + } else { + // verfiy the users account status before allowing access. + checkAccountStatus(user); + + // update the users last accessed time + user.setLastAccessed(new Date()); + } + + // persist the user's updates + UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user); + updateUser.execute(daoFactory, authorityProvider); + + // persist the user's authorities + UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user); + updateUserAuthorities.execute(daoFactory, authorityProvider); + + if (providerError != null) { + throw new AccountDisabledException(String.format("User credentials for %s were not found. This account has been disabled.", user.getDn()), providerError); + } + } + + return user; + } + + /** + * Determines if account verification is required. + * + * @return + */ + private boolean isAccountVerificationRequired(NiFiUser user) { + // accounts that have never been verified obviously needs to be re-verified + if (user.getLastVerified() == null) { + return true; + } + + // create a calendar and substract the threshold - anything + // before this time will need to be re-verified + Calendar calendar = Calendar.getInstance(); + calendar.add(Calendar.SECOND, -cacheDurationSeconds); + + return user.getLastVerified().before(calendar.getTime()); + } + + /** + * Checks the account status of the specified user. + * + * @param user + */ + private void checkAccountStatus(NiFiUser user) { + if (AccountStatus.DISABLED.equals(user.getStatus())) { + throw new AccountDisabledException(String.format("Account for %s is disabled.", user.getDn())); + } else if (AccountStatus.PENDING.equals(user.getStatus())) { + throw new AccountPendingException(String.format("Account for %s is pending.", user.getDn())); + } + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java new file mode 100644 index 0000000000..3833abb6fa --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/CreateUserAction.java @@ -0,0 +1,53 @@ +/* + * 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.admin.service.action; + +import java.util.Set; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; + +/** + * Action for creating a NiFiUser account. + */ +public class CreateUserAction extends AbstractUserAction { + + private final NiFiUser user; + + public CreateUserAction(NiFiUser user) { + this.user = user; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + AuthorityDAO authorityDao = daoFactory.getAuthorityDAO(); + + // create the user entry + userDao.createUser(user); + + // create the authorities + Set authorities = user.getAuthorities(); + authorityDao.createAuthorities(authorities, user.getId()); + + return null; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java new file mode 100644 index 0000000000..f93e97e778 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DeleteUserAction.java @@ -0,0 +1,68 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class DeleteUserAction implements AdministrationAction { + + private final String userId; + + /** + * Creates a new transactions for deleting the specified user. + * + * @param userId + */ + public DeleteUserAction(String userId) { + this.userId = userId; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + final AuthorityDAO authorityDAO = daoFactory.getAuthorityDAO(); + final UserDAO userDAO = daoFactory.getUserDAO(); + + // find the user and ensure they are currently revoked + final NiFiUser user = userDAO.findUserById(userId); + + // ensure the user was found + if (user == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", userId)); + } + + // ensure the user is in the appropriate state + if (AccountStatus.ACTIVE.equals(user.getStatus())) { + throw new IllegalStateException(String.format("An active user cannot be removed. Revoke user access before attempting to remove.")); + } + + // remove the user and their authorities + authorityDAO.deleteAuthorities(userId); + userDAO.deleteUser(userId); + + return null; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java new file mode 100644 index 0000000000..c31f107912 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserAction.java @@ -0,0 +1,76 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ +public class DisableUserAction implements AdministrationAction { + + private static final Logger logger = LoggerFactory.getLogger(DisableUserAction.class); + + private final String id; + + public DisableUserAction(String id) { + this.id = id; + } + + @Override + public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + + // get the user + NiFiUser user = userDao.findUserById(id); + + // ensure the user exists + if (user == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", id)); + } + + // update the account + user.setStatus(AccountStatus.DISABLED); + user.setUserGroup(null); + + // update the user locally + userDao.updateUser(user); + + try { + // revoke the user in the authority provider + authorityProvider.revokeUser(user.getDn()); + } catch (UnknownIdentityException uie) { + // user identity is not known + logger.info(String.format("User %s has already been removed from the authority provider.", user.getDn())); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to revoke user '%s': %s", user.getDn(), aae.getMessage()), aae); + } + + return user; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java new file mode 100644 index 0000000000..385fce6bb2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/DisableUserGroupAction.java @@ -0,0 +1,69 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUserGroup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ +public class DisableUserGroupAction implements AdministrationAction { + + private static final Logger logger = LoggerFactory.getLogger(DisableUserGroupAction.class); + + private final String group; + + public DisableUserGroupAction(final String group) { + this.group = group; + } + + @Override + public NiFiUserGroup execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + final NiFiUserGroup userGroup = new NiFiUserGroup(); + + final UserDAO userDao = daoFactory.getUserDAO(); + + // update the user group locally + userDao.updateGroupStatus(group, AccountStatus.DISABLED); + + // populate the group details + userGroup.setGroup(group); + userGroup.setUsers(userDao.findUsersForGroup(group)); + + try { + // revoke the user in the authority provider + authorityProvider.revokeGroup(group); + } catch (UnknownIdentityException uie) { + // user identity is not known + logger.info(String.format("User group %s has already been removed from the authority provider.", group)); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to revoke user group '%s': %s", group, aae.getMessage()), aae); + } + + return userGroup; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java new file mode 100644 index 0000000000..8e5b574d50 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByDnAction.java @@ -0,0 +1,49 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class FindUserByDnAction implements AdministrationAction { + + private final String dn; + + /** + * Creates a new transactions for getting a user with the specified DN. + * + * @param dn The DN of the user to obtain + */ + public FindUserByDnAction(String dn) { + this.dn = dn; + } + + @Override + public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + // get a UserDAO + UserDAO userDAO = daoFactory.getUserDAO(); + + // return the desired user + return userDAO.findUserByDn(dn); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java new file mode 100644 index 0000000000..3062a2ea75 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/FindUserByIdAction.java @@ -0,0 +1,49 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class FindUserByIdAction implements AdministrationAction { + + private final String id; + + /** + * Creates a new transactions for getting a user with the specified id. + * + * @param id + */ + public FindUserByIdAction(String id) { + this.id = id; + } + + @Override + public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + // get a UserDAO + UserDAO userDAO = daoFactory.getUserDAO(); + + // return the desired user + return userDAO.findUserById(id); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java new file mode 100644 index 0000000000..1dc558840f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionAction.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.action; + +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * Gets the action with the specified id. + */ +public class GetActionAction implements AdministrationAction { + + private final Integer id; + + public GetActionAction(Integer id) { + this.id = id; + } + + @Override + public Action execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + ActionDAO actionDao = daoFactory.getActionDAO(); + return actionDao.getAction(id); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java new file mode 100644 index 0000000000..3b82d7985f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetActionsAction.java @@ -0,0 +1,48 @@ +/* + * 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.admin.service.action; + +import java.util.Date; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; + +/** + * Get all actions that match the specified query. + */ +public class GetActionsAction implements AdministrationAction { + + private final HistoryQuery query; + + public GetActionsAction(HistoryQuery query) { + this.query = query; + } + + @Override + public History execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + ActionDAO actionDao = daoFactory.getActionDAO(); + + // find all matching history + History history = actionDao.findActions(query); + history.setLastRefreshed(new Date()); + + return history; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java new file mode 100644 index 0000000000..5ce663eb39 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetPreviousValues.java @@ -0,0 +1,43 @@ +/* + * 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.admin.service.action; + +import java.util.List; +import java.util.Map; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.history.PreviousValue; + +/** + * Gets the action with the specified id. + */ +public class GetPreviousValues implements AdministrationAction>> { + + private final String processorId; + + public GetPreviousValues(String processorId) { + this.processorId = processorId; + } + + @Override + public Map> execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + ActionDAO actionDao = daoFactory.getActionDAO(); + return actionDao.getPreviousValues(processorId); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java new file mode 100644 index 0000000000..5377c4650b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUserGroupAction.java @@ -0,0 +1,50 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUserGroup; + +/** + * + */ +public class GetUserGroupAction implements AdministrationAction { + + private final String group; + + public GetUserGroupAction(String group) { + this.group = group; + } + + @Override + public NiFiUserGroup execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + final UserDAO userDAO = daoFactory.getUserDAO(); + final NiFiUserGroup userGroup = new NiFiUserGroup(); + + // set the group + userGroup.setGroup(group); + + // get the users in this group + userGroup.setUsers(userDAO.findUsersForGroup(group)); + + // return the group + return userGroup; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java new file mode 100644 index 0000000000..42d180e5ac --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/GetUsersAction.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.action; + +import java.util.Collection; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class GetUsersAction implements AdministrationAction> { + + @Override + public Collection execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + // get a UserDAO + UserDAO userDAO = daoFactory.getUserDAO(); + + // return the desired user + return userDAO.findUsers(); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java new file mode 100644 index 0000000000..3325642fb1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/HasPendingUserAccounts.java @@ -0,0 +1,34 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * Action for creating a NiFiUser account. + */ +public class HasPendingUserAccounts extends AbstractUserAction { + + @Override + public Boolean execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + return userDao.hasPendingUserAccounts(); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java new file mode 100644 index 0000000000..14596b2bf1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserAccountAction.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; + +/** + * Invalidates a user account. + */ +public class InvalidateUserAccountAction implements AdministrationAction { + + private final String id; + + public InvalidateUserAccountAction(String id) { + this.id = id; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + + // get the current user details + NiFiUser user = userDao.findUserById(id); + + // ensure the user exists + if (user == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", id)); + } + + // invalidate the user account + user.setLastVerified(null); + + // create the user entry + userDao.updateUser(user); + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java new file mode 100644 index 0000000000..0cb7e144b2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/InvalidateUserGroupAccountsAction.java @@ -0,0 +1,45 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * Invalidates a user account. + */ +public class InvalidateUserGroupAccountsAction implements AdministrationAction { + + private final String group; + + public InvalidateUserGroupAccountsAction(String group) { + this.group = group; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + + // create the user entry + userDao.updateGroupVerification(group, null); + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java new file mode 100644 index 0000000000..b5a2883720 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/PurgeActionsAction.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.action; + +import java.util.Date; +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.dao.ActionDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * Purges actions up to a specified end date. + */ +public class PurgeActionsAction implements AdministrationAction { + + private final Date end; + private final Action purgeAction; + + public PurgeActionsAction(Date end, Action purgeAction) { + this.end = end; + this.purgeAction = purgeAction; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + ActionDAO actionDao = daoFactory.getActionDAO(); + + // remove the corresponding actions + actionDao.deleteActions(end); + + // create a purge action + actionDao.createAction(purgeAction); + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java new file mode 100644 index 0000000000..3dce6d9e2e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/RequestUserAccountAction.java @@ -0,0 +1,67 @@ +/* + * 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.admin.service.action; + +import java.util.Date; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.security.util.CertificateUtils; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class RequestUserAccountAction implements AdministrationAction { + + private final String dn; + private final String justification; + + public RequestUserAccountAction(String dn, String justification) { + this.dn = dn; + this.justification = justification; + } + + @Override + public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + + // determine if this user already exists + NiFiUser user = userDao.findUserByDn(dn); + if (user != null) { + throw new IllegalArgumentException(String.format("User account for %s already exists.", dn)); + } + + // create the user + user = new NiFiUser(); + user.setDn(dn); + user.setUserName(CertificateUtils.extractUsername(dn)); + user.setJustification(justification); + user.setStatus(AccountStatus.PENDING); + + // update user timestamps + Date now = new Date(); + user.setCreation(now); + + // create the new user account + userDao.createUser(user); + + return user; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.java new file mode 100644 index 0000000000..72d68db2dc --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/SeedUserAccountsAction.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.admin.service.action; + +import java.util.HashSet; +import java.util.Set; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.security.util.CertificateUtils; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Seeds the user accounts. This action is performed at start up because it + * takes the users specified in the authority provider and makes them available + * to be seen in the UI. This happens because the UI loads the users from the + * cache. Without pre loading the users, the table in the UI would only show a + * given user once they have visited the application. + */ +public class SeedUserAccountsAction extends AbstractUserAction { + + private static final Logger logger = LoggerFactory.getLogger(SeedUserAccountsAction.class); + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + Set authorizedDns = new HashSet<>(); + + // get the current user cache + final Set existingUsers; + try { + existingUsers = userDao.findUsers(); + } catch (Exception e) { + // unable to access local cache... start up failure + logger.error(String.format("Unable to get existing user base. Cannot proceed until these users can be " + + "verified against the current authority provider: %s", e)); + throw new AdministrationException(e); + } + + try { + // all users for all roles + for (final Authority authority : Authority.values()) { + authorizedDns.addAll(authorityProvider.getUsers(authority)); + } + } catch (AuthorityAccessException aae) { + // unable to access the authority provider... honor the cache + logger.warn("Unable to access authority provider due to " + aae); + return null; + } + + final Set accountsToRevoke = new HashSet<>(existingUsers); + + // persist the users + for (String dn : authorizedDns) { + NiFiUser user = null; + try { + // locate the user for this dn + user = userDao.findUserByDn(dn); + boolean newAccount = false; + + // if the user does not exist, create a new account + if (user == null) { + logger.info(String.format("Creating user account: %s", dn)); + newAccount = true; + + // create the user + user = new NiFiUser(); + user.setDn(dn); + user.setUserName(CertificateUtils.extractUsername(dn)); + user.setJustification("User details specified by authority provider."); + } else { + logger.info(String.format("User account already created: %s. Updating authorities...", dn)); + } + + // verify the account + verifyAccount(authorityProvider, user); + + // persist the account accordingly + if (newAccount) { + CreateUserAction createUser = new CreateUserAction(user); + createUser.execute(daoFactory, authorityProvider); + } else { + // this is not a new user and we have just verified their + // account, do not revoke... + accountsToRevoke.remove(user); + + // persist the user + UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user); + updateUser.execute(daoFactory, authorityProvider); + + // persist the user's authorities + UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user); + updateUserAuthorities.execute(daoFactory, authorityProvider); + } + } catch (DataAccessException dae) { + if (user != null) { + logger.warn(String.format("Unable to access account details in local cache for user %s: %s", user, dae.getMessage())); + } else { + logger.warn(String.format("Unable to access account details in local cache: %s", dae.getMessage())); + } + } catch (UnknownIdentityException uie) { + if (user != null) { + logger.warn(String.format("Unable to find account details in authority provider for user %s: %s", user, uie.getMessage())); + } else { + logger.warn(String.format("Unable to find account details in authority provider: %s", uie.getMessage())); + } + } catch (AuthorityAccessException aae) { + logger.warn("Unable to access authority provider due to " + aae); + + // unable to access authority provider for this user, honor the cache for now + accountsToRevoke.remove(user); + } + } + + // remove all users that are no longer in the provider + for (final NiFiUser user : accountsToRevoke) { + // allow pending requests to remain... + if (AccountStatus.PENDING.equals(user.getStatus())) { + continue; + } + + try { + logger.info(String.format("User not authorized with configured provider: %s. Disabling account...", user.getDn())); + + // disable the account and reset its last verified timestamp since it was not found + // in the current configured authority provider + user.setStatus(AccountStatus.DISABLED); + user.setLastVerified(null); + + // update the user record + UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user); + updateUser.execute(daoFactory, authorityProvider); + } catch (final Exception e) { + // unable to revoke access for someone we know is not authorized... fail start up + logger.error(String.format("Unable to revoke access for user %s that is no longer authorized: %s", user, e)); + throw new AdministrationException(e); + } + } + + return null; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java new file mode 100644 index 0000000000..01eaf5f1c7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserAction.java @@ -0,0 +1,69 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.user.NiFiUser; + +/** + * + */ +public class UngroupUserAction extends AbstractUserAction { + + private final String userId; + + public UngroupUserAction(String userId) { + this.userId = userId; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + final UserDAO userDao = daoFactory.getUserDAO(); + + // get the user in question + final NiFiUser user = userDao.findUserById(userId); + + // ensure the user exists + if (user == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", userId)); + } + + // set the user group + user.setUserGroup(null); + + // update the user locally + userDao.updateUser(user); + + try { + // update the authority provider + authorityProvider.ungroupUser(user.getDn()); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to ungroup user '%s': %s", user.getDn(), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to ungroup user '%s': %s", user.getDn(), aae.getMessage()), aae); + } + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.java new file mode 100644 index 0000000000..fa24fbeb64 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UngroupUserGroupAction.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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; + +/** + * + */ +public class UngroupUserGroupAction extends AbstractUserAction { + + private final String group; + + public UngroupUserGroupAction(String group) { + this.group = group; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) { + final UserDAO userDao = daoFactory.getUserDAO(); + + // update the user locally + userDao.ungroup(group); + + try { + // update the authority provider + authorityProvider.ungroup(group); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to ungroup '%s': %s", group, uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to ungroup '%s': %s", group, aae.getMessage()), aae); + } + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java new file mode 100644 index 0000000000..cef21d7d97 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAction.java @@ -0,0 +1,124 @@ +/* + * 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.admin.service.action; + +import java.util.Date; +import java.util.Set; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Sets user authorities. + */ +public class UpdateUserAction extends AbstractUserAction { + + private static final Logger logger = LoggerFactory.getLogger(UpdateUserAction.class); + + private final String id; + private final Set authorities; + + public UpdateUserAction(String id, Set authorities) { + this.id = id; + this.authorities = authorities; + } + + @Override + public NiFiUser execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException, AdministrationException { + UserDAO userDao = daoFactory.getUserDAO(); + + // get the user + NiFiUser user = userDao.findUserById(id); + + // ensure the user exists + if (user == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", id)); + } + + // determine whether this users exists + boolean doesDnExist = false; + try { + doesDnExist = authorityProvider.doesDnExist(user.getDn()); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae); + } + + // if the user already doesn't exist, add them + if (!doesDnExist) { + try { + // add the account account and group if necessary + authorityProvider.addUser(user.getDn(), user.getUserGroup()); + } catch (final IdentityAlreadyExistsException iaee) { + logger.warn(String.format("User '%s' already exists in the authority provider. Continuing with user update.", user.getDn())); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authorities for '%s': %s", user.getDn(), aae.getMessage()), aae); + } + } + + try { + // update the authority provider as approprivate + authorityProvider.setAuthorities(user.getDn(), authorities); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae); + } + + try { + // get the user group + user.setUserGroup(authorityProvider.getGroupForUser(user.getDn())); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getDn(), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getDn(), aae.getMessage()), aae); + } + + // since all the authorities were updated accordingly, set the authorities + user.getAuthorities().clear(); + user.getAuthorities().addAll(authorities); + + // update the users status in case they were previously pending or disabled + user.setStatus(AccountStatus.ACTIVE); + + // update the users last verified time - this timestamp shouldn't be recorded + // until the both the user's authorities and group have been synced + Date now = new Date(); + user.setLastVerified(now); + + // persist the user's updates + UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user); + updateUser.execute(daoFactory, authorityProvider); + + // persist the user's authorities + UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user); + updateUserAuthorities.execute(daoFactory, authorityProvider); + + // return the user + return user; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java new file mode 100644 index 0000000000..89661b20a6 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserAuthoritiesCacheAction.java @@ -0,0 +1,73 @@ +/* + * 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.admin.service.action; + +import java.util.Set; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.collections4.CollectionUtils; + +/** + * Updates a NiFiUser's authorities. Prior to invoking this action, the user's + * authorities should be set according to the business logic of the service in + * question. This should not be invoked directly when attempting to set user + * authorities as the authorityProvider is not called from this action. + */ +public class UpdateUserAuthoritiesCacheAction extends AbstractUserAction { + + private final NiFiUser user; + + public UpdateUserAuthoritiesCacheAction(NiFiUser user) { + this.user = user; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + AuthorityDAO authorityDao = daoFactory.getAuthorityDAO(); + + // get the user + NiFiUser currentUser = userDao.findUserById(user.getId()); + + // ensure the user exists + if (currentUser == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", user.getId())); + } + + // determine what authorities need to be added/removed + Set authorities = user.getAuthorities(); + Set authoritiesToAdd = determineAuthoritiesToAdd(currentUser, authorities); + Set authoritiesToRemove = determineAuthoritiesToRemove(currentUser, authorities); + + // update the user authorities locally + if (CollectionUtils.isNotEmpty(authoritiesToAdd)) { + authorityDao.createAuthorities(authoritiesToAdd, user.getId()); + } + if (CollectionUtils.isNotEmpty(authoritiesToRemove)) { + authorityDao.deleteAuthorities(authoritiesToRemove, user.getId()); + } + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java new file mode 100644 index 0000000000..288e2975d4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserCacheAction.java @@ -0,0 +1,47 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.NiFiUser; + +/** + * Updates a NiFiUser. This will not update the user authorities, they must be + * updated with the UpdateUserAuthoritiesAction. + */ +public class UpdateUserCacheAction extends AbstractUserAction { + + private final NiFiUser user; + + public UpdateUserCacheAction(NiFiUser user) { + this.user = user; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + UserDAO userDao = daoFactory.getUserDAO(); + + // update the user + userDao.updateUser(user); + + return null; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java new file mode 100644 index 0000000000..56b214cd95 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/action/UpdateUserGroupAction.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.action; + +import java.util.Date; +import java.util.HashSet; +import java.util.Set; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Updates all NiFiUser authorities in a specified group. + */ +public class UpdateUserGroupAction extends AbstractUserAction { + + private static final Logger logger = LoggerFactory.getLogger(UpdateUserGroupAction.class); + + private final String group; + private final Set userIds; + private final Set authorities; + + public UpdateUserGroupAction(String group, Set userIds, Set authorities) { + this.group = group; + this.userIds = userIds; + this.authorities = authorities; + } + + @Override + public Void execute(DAOFactory daoFactory, AuthorityProvider authorityProvider) throws DataAccessException { + if (userIds == null && authorities == null) { + throw new IllegalArgumentException("Must specify user Ids or authorities."); + } + + UserDAO userDao = daoFactory.getUserDAO(); + + // record the new users being added to this group + final Set newUsers = new HashSet<>(); + final Set newUserDns = new HashSet<>(); + + // if the user ids have been specified we need to create/update a group using the specified group name + if (userIds != null) { + if (userIds.isEmpty()) { + throw new IllegalArgumentException("When creating a group, at least one user id must be specified."); + } + + // going to create a group using the specified user ids + for (final String userId : userIds) { + // get the user in question + final NiFiUser user = userDao.findUserById(userId); + + // ensure the user exists + if (user == null) { + throw new AccountNotFoundException(String.format("Unable to find account with ID %s.", userId)); + } + + try { + // if the user is unknown to the authority provider we cannot continue + if (!authorityProvider.doesDnExist(user.getDn()) || AccountStatus.DISABLED.equals(user.getStatus())) { + throw new IllegalStateException(String.format("Unable to group these users because access for '%s' is not %s.", user.getDn(), AccountStatus.ACTIVE.toString())); + } + + // record the user being added to this group + newUsers.add(user); + newUserDns.add(user.getDn()); + } catch (final AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authority details: %s", aae.getMessage()), aae); + } + } + + try { + // update the authority provider + authorityProvider.setUsersGroup(newUserDns, group); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserDns, ", "), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to set user group '%s': %s", StringUtils.join(newUserDns, ", "), aae.getMessage()), aae); + } + } + + // get all the users that need to be updated + final Set users = new HashSet<>(userDao.findUsersForGroup(group)); + users.addAll(newUsers); + + // ensure the user exists + if (users.isEmpty()) { + throw new AccountNotFoundException(String.format("Unable to find user accounts with group id %s.", group)); + } + + // update each user in this group + for (final NiFiUser user : users) { + // if there are new authorities set them, otherwise refresh them according to the provider + if (authorities != null) { + try { + // update the authority provider as approprivate + authorityProvider.setAuthorities(user.getDn(), authorities); + + // since all the authorities were updated accordingly, set the authorities + user.getAuthorities().clear(); + user.getAuthorities().addAll(authorities); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to modify authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae); + } + } else { + try { + // refresh the authorities according to the provider + user.getAuthorities().clear(); + user.getAuthorities().addAll(authorityProvider.getAuthorities(user.getDn())); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to determine the authorities for '%s': %s.", user.getDn(), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access authorities for '%s': %s.", user.getDn(), aae.getMessage()), aae); + } + } + + try { + // get the user group + user.setUserGroup(authorityProvider.getGroupForUser(user.getDn())); + } catch (UnknownIdentityException uie) { + throw new AccountNotFoundException(String.format("Unable to determine the group for '%s': %s.", user.getDn(), uie.getMessage()), uie); + } catch (AuthorityAccessException aae) { + throw new AdministrationException(String.format("Unable to access the group for '%s': %s.", user.getDn(), aae.getMessage()), aae); + } + + // update the users status in case they were previously pending or disabled + user.setStatus(AccountStatus.ACTIVE); + + // update the users last verified time - this timestamp shouldn't be recorded + // until the both the user's authorities and group have been synced + Date now = new Date(); + user.setLastVerified(now); + + // persist the user's updates + UpdateUserCacheAction updateUser = new UpdateUserCacheAction(user); + updateUser.execute(daoFactory, authorityProvider); + + // persist the user's authorities + UpdateUserAuthoritiesCacheAction updateUserAuthorities = new UpdateUserAuthoritiesCacheAction(user); + updateUserAuthorities.execute(daoFactory, authorityProvider); + } + + return null; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java new file mode 100644 index 0000000000..127f1df813 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardAuditService.java @@ -0,0 +1,230 @@ +/* + * 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.admin.service.impl; + +import java.io.IOException; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.nifi.action.Action; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.admin.service.action.AddActionsAction; +import org.apache.nifi.admin.service.action.GetActionAction; +import org.apache.nifi.admin.service.action.GetActionsAction; +import org.apache.nifi.admin.service.action.GetPreviousValues; +import org.apache.nifi.admin.service.action.PurgeActionsAction; +import org.apache.nifi.admin.service.transaction.Transaction; +import org.apache.nifi.admin.service.transaction.TransactionBuilder; +import org.apache.nifi.admin.service.transaction.TransactionException; +import org.apache.nifi.history.History; +import org.apache.nifi.history.HistoryQuery; +import org.apache.nifi.history.PreviousValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ +public class StandardAuditService implements AuditService { + + private static final Logger logger = LoggerFactory.getLogger(StandardAuditService.class); + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); + private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); + + private TransactionBuilder transactionBuilder; + + @Override + public void addActions(Collection actions) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // seed the accounts + AddActionsAction addActions = new AddActionsAction(actions); + transaction.execute(addActions); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public Map> getPreviousValues(String processorId) { + Transaction transaction = null; + Map> previousValues = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // seed the accounts + GetPreviousValues getActions = new GetPreviousValues(processorId); + previousValues = transaction.execute(getActions); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + + return previousValues; + } + + @Override + public History getActions(HistoryQuery query) { + Transaction transaction = null; + History history = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // seed the accounts + GetActionsAction getActions = new GetActionsAction(query); + history = transaction.execute(getActions); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + + return history; + } + + @Override + public Action getAction(Integer actionId) { + Transaction transaction = null; + Action action = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // seed the accounts + GetActionAction getAction = new GetActionAction(actionId); + action = transaction.execute(getAction); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + + return action; + } + + @Override + public void purgeActions(Date end, Action purgeAction) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // purge the action database + PurgeActionsAction purgeActions = new PurgeActionsAction(end, purgeAction); + transaction.execute(purgeActions); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + /** + * Rolls back the specified transaction. + * + * @param transaction + */ + private void rollback(Transaction transaction) { + if (transaction != null) { + transaction.rollback(); + } + } + + /** + * Closes the specified transaction. + * + * @param transaction + */ + private void closeQuietly(final Transaction transaction) { + if (transaction != null) { + try { + transaction.close(); + } catch (final IOException ioe) { + } + } + } + + /* setters */ + public void setTransactionBuilder(TransactionBuilder transactionBuilder) { + this.transactionBuilder = transactionBuilder; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java new file mode 100644 index 0000000000..63aa93bafb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/impl/StandardUserService.java @@ -0,0 +1,629 @@ +/* + * 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.admin.service.impl; + +import java.io.IOException; +import java.util.Collection; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.service.AccountDisabledException; +import org.apache.nifi.admin.service.AccountPendingException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.admin.service.UserService; +import org.apache.nifi.admin.service.action.AuthorizeUserAction; +import org.apache.nifi.admin.service.action.DeleteUserAction; +import org.apache.nifi.admin.service.action.DisableUserAction; +import org.apache.nifi.admin.service.action.DisableUserGroupAction; +import org.apache.nifi.admin.service.action.FindUserByDnAction; +import org.apache.nifi.admin.service.action.FindUserByIdAction; +import org.apache.nifi.admin.service.action.GetUserGroupAction; +import org.apache.nifi.admin.service.action.GetUsersAction; +import org.apache.nifi.admin.service.action.HasPendingUserAccounts; +import org.apache.nifi.admin.service.action.InvalidateUserAccountAction; +import org.apache.nifi.admin.service.action.InvalidateUserGroupAccountsAction; +import org.apache.nifi.admin.service.action.RequestUserAccountAction; +import org.apache.nifi.admin.service.action.SeedUserAccountsAction; +import org.apache.nifi.admin.service.action.UpdateUserAction; +import org.apache.nifi.admin.service.action.UpdateUserGroupAction; +import org.apache.nifi.admin.service.action.UngroupUserAction; +import org.apache.nifi.admin.service.action.UngroupUserGroupAction; +import org.apache.nifi.admin.service.transaction.Transaction; +import org.apache.nifi.admin.service.transaction.TransactionBuilder; +import org.apache.nifi.admin.service.transaction.TransactionException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.user.NiFiUser; +import org.apache.nifi.user.NiFiUserGroup; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + */ +public class StandardUserService implements UserService { + + private static final Logger logger = LoggerFactory.getLogger(StandardUserService.class); + + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); + private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); + private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); + + private TransactionBuilder transactionBuilder; + private NiFiProperties properties; + + /** + * Seed any users from the authority provider that are not already present. + */ + public void seedUserAccounts() { + // do not seed node's user cache. when/if the node disconnects its + // cache will be populated lazily (as needed) + if (properties.isNode()) { + return; + } + + Transaction transaction = null; + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // seed the accounts + SeedUserAccountsAction seedUserAccounts = new SeedUserAccountsAction(); + transaction.execute(seedUserAccounts); + + // commit the transaction + transaction.commit(); + } catch (AdministrationException ae) { + rollback(transaction); + throw ae; + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public NiFiUser createPendingUserAccount(String dn, String justification) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // create the account request + RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(dn, justification); + NiFiUser user = transaction.execute(requestUserAccount); + + // commit the transaction + transaction.commit(); + + // return the nifi user + return user; + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public NiFiUserGroup updateGroup(final String group, final Set userIds, final Set authorities) { + Transaction transaction = null; + + writeLock.lock(); + try { + // if user ids have been specified, invalidate the user accounts before performing + // the desired updates. if case of an error, this will ensure that these users are + // authorized the next time the access the application + if (userIds != null) { + for (final String userId : userIds) { + invalidateUserAccount(userId); + } + } + + // start the transaction + transaction = transactionBuilder.start(); + + // set the authorities for each user in this group if specified + final UpdateUserGroupAction updateUserGroup = new UpdateUserGroupAction(group, userIds, authorities); + transaction.execute(updateUserGroup); + + // get all the users that are now in this group + final GetUserGroupAction getUserGroup = new GetUserGroupAction(group); + final NiFiUserGroup userGroup = transaction.execute(getUserGroup); + + // commit the transaction + transaction.commit(); + + return userGroup; + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public void ungroupUser(String id) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // ungroup the specified user + final UngroupUserAction ungroupUser = new UngroupUserAction(id); + transaction.execute(ungroupUser); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public void ungroup(String group) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // ungroup the specified user + final UngroupUserGroupAction ungroupUserGroup = new UngroupUserGroupAction(group); + transaction.execute(ungroupUserGroup); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public NiFiUser checkAuthorization(String dn) { + Transaction transaction = null; + + writeLock.lock(); + try { + // create the connection + transaction = transactionBuilder.start(); + + // determine how long the cache is valid for + final int cacheSeconds; + try { + cacheSeconds = (int) FormatUtils.getTimeDuration(properties.getUserCredentialCacheDuration(), TimeUnit.SECONDS); + } catch (IllegalArgumentException iae) { + throw new AdministrationException("User credential cache duration is not configured correctly."); + } + + // attempt to authorize the user + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(dn, cacheSeconds); + NiFiUser user = transaction.execute(authorizeUser); + + // commit the transaction + transaction.commit(); + + // return the nifi user + return user; + } catch (DataAccessException | TransactionException dae) { + rollback(transaction); + throw new AdministrationException(dae); + } catch (AccountDisabledException | AccountPendingException ade) { + rollback(transaction); + throw ade; + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public void deleteUser(String id) { + Transaction transaction = null; + + writeLock.lock(); + try { + // create the connection + transaction = transactionBuilder.start(); + + // delete the user + DeleteUserAction deleteUser = new DeleteUserAction(id); + transaction.execute(deleteUser); + + // commit the transaction + transaction.commit(); + } catch (DataAccessException | TransactionException dae) { + rollback(transaction); + throw new AdministrationException(dae); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public NiFiUser disable(String id) { + Transaction transaction = null; + + writeLock.lock(); + try { + // create the connection + transaction = transactionBuilder.start(); + + // disable the user + DisableUserAction disableUser = new DisableUserAction(id); + NiFiUser user = transaction.execute(disableUser); + + // commit the transaction + transaction.commit(); + + // return the user + return user; + } catch (DataAccessException | TransactionException dae) { + rollback(transaction); + throw new AdministrationException(dae); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public NiFiUserGroup disableGroup(String group) { + Transaction transaction = null; + + writeLock.lock(); + try { + // create the connection + transaction = transactionBuilder.start(); + + // disable the user + DisableUserGroupAction disableUser = new DisableUserGroupAction(group); + NiFiUserGroup userGroup = transaction.execute(disableUser); + + // commit the transaction + transaction.commit(); + + // return the user + return userGroup; + } catch (DataAccessException | TransactionException dae) { + rollback(transaction); + throw new AdministrationException(dae); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + @Override + public NiFiUser update(String id, Set authorities) { + Transaction transaction = null; + + // may be empty but not null + if (authorities == null) { + throw new IllegalArgumentException("The specified authorities cannot be null."); + } + + writeLock.lock(); + try { + // invalidate the user account in preparation for potential subsequent errors + invalidateUserAccount(id); + + // at this point the current user account has been invalidated so we will + // attempt to update the account. if any part fails we are assured the + // user will be need to be given approval before they access the system at + // a later time + // start the transaction + transaction = transactionBuilder.start(); + + // update the user authorities + UpdateUserAction setUserAuthorities = new UpdateUserAction(id, authorities); + NiFiUser user = transaction.execute(setUserAuthorities); + + // commit the transaction + transaction.commit(); + + // return the user + return user; + } catch (TransactionException | DataAccessException e) { + rollback(transaction); + throw new AdministrationException(e); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + /** + * Invalidates the user with the specified id. This is done to ensure a user + * account will need to be re-validated in case an error occurs while + * modifying a user account. This method should only be invoked from within + * a write lock. + * + * @param id + */ + @Override + public void invalidateUserAccount(String id) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // invalidate the user account + InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(id); + transaction.execute(invalidateUserAccount); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + /** + * Invalidates the user with the specified id. This is done to ensure a user + * account will need to be re-validated in case an error occurs while + * modifying a user account. This method should only be invoked from within + * a write lock. + * + * @param id + */ + @Override + public void invalidateUserGroupAccount(String group) { + Transaction transaction = null; + + writeLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // invalidate the user account + InvalidateUserGroupAccountsAction invalidateUserGroupAccounts = new InvalidateUserGroupAccountsAction(group); + transaction.execute(invalidateUserGroupAccounts); + + // commit the transaction + transaction.commit(); + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + writeLock.unlock(); + } + } + + // ----------------- + // read only methods + // ----------------- + @Override + public Boolean hasPendingUserAccount() { + Transaction transaction = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + final HasPendingUserAccounts hasPendingAccounts = new HasPendingUserAccounts(); + final Boolean hasPendingUserAccounts = transaction.execute(hasPendingAccounts); + + // commit the transaction + transaction.commit(); + + return hasPendingUserAccounts; + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + } + + @Override + public Collection getUsers() { + Transaction transaction = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // get all users + GetUsersAction getUsers = new GetUsersAction(); + Collection users = transaction.execute(getUsers); + + // commit the transaction + transaction.commit(); + + // return the users + return users; + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + } + + @Override + public NiFiUser getUserById(String id) { + Transaction transaction = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // return the desired user + FindUserByIdAction findUserById = new FindUserByIdAction(id); + NiFiUser user = transaction.execute(findUserById); + + // commit the transaction + transaction.commit(); + + // return the user + return user; + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + } + + @Override + public NiFiUser getUserByDn(String dn) { + Transaction transaction = null; + + readLock.lock(); + try { + // start the transaction + transaction = transactionBuilder.start(); + + // return the desired user + FindUserByDnAction findUserByDn = new FindUserByDnAction(dn); + NiFiUser user = transaction.execute(findUserByDn); + + // commit the transaction + transaction.commit(); + + // return the user + return user; + } catch (TransactionException | DataAccessException te) { + rollback(transaction); + throw new AdministrationException(te); + } catch (Throwable t) { + rollback(transaction); + throw t; + } finally { + closeQuietly(transaction); + readLock.unlock(); + } + } + + /** + * Rolls back the specified transaction. + * + * @param transaction + */ + private void rollback(final Transaction transaction) { + if (transaction != null) { + transaction.rollback(); + } + } + + /** + * Closes the specified transaction. + * + * @param transaction + */ + private void closeQuietly(final Transaction transaction) { + if (transaction != null) { + try { + transaction.close(); + } catch (final IOException ioe) { + } + } + } + + /* + * setters + */ + public void setTransactionBuilder(TransactionBuilder transactionBuilder) { + this.transactionBuilder = transactionBuilder; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java new file mode 100644 index 0000000000..edd214ba9c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/Transaction.java @@ -0,0 +1,49 @@ +/* + * 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.admin.service.transaction; + +import java.io.Closeable; +import org.apache.nifi.admin.service.action.AdministrationAction; + +/** + * Defines a transaction. + */ +public interface Transaction extends Closeable { + + /** + * Executes the specified action within the current transaction. + * + * @param + * @param action + * @return + * @throws IllegalStateException - if there is no current transaction + */ + T execute(AdministrationAction action); + + /** + * Commits the current transaction. + * + * @throws TransactionException - if the transaction is unable to be + * committed + */ + void commit() throws TransactionException; + + /** + * Rolls back the current transaction. + */ + void rollback(); +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java new file mode 100644 index 0000000000..2d2ef82eb2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionBuilder.java @@ -0,0 +1,25 @@ +/* + * 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.admin.service.transaction; + +/** + * + */ +public interface TransactionBuilder { + + Transaction start() throws TransactionException; +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java new file mode 100644 index 0000000000..924e01f201 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/TransactionException.java @@ -0,0 +1,40 @@ +/* + * 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.admin.service.transaction; + +/** + * Exception to indicate that the user account is disabled. + */ +public class TransactionException extends RuntimeException { + + public TransactionException(String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } + + public TransactionException(Throwable cause) { + super(cause); + } + + public TransactionException(String message, Throwable cause) { + super(message, cause); + } + + public TransactionException(String message) { + super(message); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java new file mode 100644 index 0000000000..a3cfb5e3c0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransaction.java @@ -0,0 +1,93 @@ +/* + * 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.admin.service.transaction.impl; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.SQLException; +import org.apache.nifi.admin.RepositoryUtils; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.impl.DAOFactoryImpl; +import org.apache.nifi.admin.service.action.AdministrationAction; +import org.apache.nifi.admin.service.transaction.TransactionException; +import org.apache.nifi.admin.service.transaction.Transaction; +import org.apache.nifi.authorization.AuthorityProvider; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Transaction implementation that uses the specified SQL Connection and + * AuthorityProvider. + */ +public class StandardTransaction implements Transaction { + + private static final Logger logger = LoggerFactory.getLogger(StandardTransaction.class); + + private final AuthorityProvider authorityProvider; + private Connection connection; + + public StandardTransaction(AuthorityProvider authorityProvider, Connection connection) { + this.authorityProvider = authorityProvider; + this.connection = connection; + } + + @Override + public T execute(AdministrationAction action) { + // ensure the transaction has been started + if (connection == null) { + throw new IllegalStateException("This transaction is not active."); + } + + // create a dao factory + DAOFactory daoFactory = new DAOFactoryImpl(connection); + + // execute the specified action + return action.execute(daoFactory, authorityProvider); + } + + @Override + public void commit() throws TransactionException { + // ensure there is an active transaction + if (connection == null) { + throw new IllegalStateException("No active transaction."); + } + + try { + // commit the transaction + connection.commit(); + } catch (SQLException sqle) { + throw new TransactionException(sqle.getMessage()); + } + } + + @Override + public void rollback() { + // ensure there is an active transaction + if (connection != null) { + // rollback the transaction + RepositoryUtils.rollback(connection, logger); + } + } + + @Override + public void close() throws IOException { + if (connection != null) { + RepositoryUtils.closeQuietly(connection); + connection = null; + } + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.java new file mode 100644 index 0000000000..b6e5a30745 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/admin/service/transaction/impl/StandardTransactionBuilder.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.admin.service.transaction.impl; + +import java.sql.Connection; +import java.sql.SQLException; +import javax.sql.DataSource; +import org.apache.nifi.admin.service.transaction.Transaction; +import org.apache.nifi.admin.service.transaction.TransactionBuilder; +import org.apache.nifi.admin.service.transaction.TransactionException; +import org.apache.nifi.authorization.AuthorityProvider; + +/** + * + */ +public class StandardTransactionBuilder implements TransactionBuilder { + + private DataSource dataSource; + private AuthorityProvider authorityProvider; + + @Override + public Transaction start() throws TransactionException { + try { + // get a new connection + Connection connection = dataSource.getConnection(); + connection.setAutoCommit(false); + + // create a new transaction + return new StandardTransaction(authorityProvider, connection); + } catch (SQLException sqle) { + throw new TransactionException(sqle.getMessage()); + } + } + + /* setters */ + public void setDataSource(DataSource dataSource) { + this.dataSource = dataSource; + } + + public void setAuthorityProvider(AuthorityProvider authorityProvider) { + this.authorityProvider = authorityProvider; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java new file mode 100644 index 0000000000..b05d32fa9c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/AuthorityProviderFactoryBean.java @@ -0,0 +1,516 @@ +/* + * 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.authorization; + +import java.io.File; +import java.lang.reflect.Constructor; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.xml.XMLConstants; +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBElement; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Unmarshaller; +import javax.xml.transform.stream.StreamSource; +import javax.xml.validation.Schema; +import javax.xml.validation.SchemaFactory; +import org.apache.nifi.authorization.annotation.AuthorityProviderContext; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException; +import org.apache.nifi.authorization.exception.ProviderCreationException; +import org.apache.nifi.authorization.exception.ProviderDestructionException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.authorization.generated.AuthorityProviders; +import org.apache.nifi.authorization.generated.Property; +import org.apache.nifi.authorization.generated.Provider; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.util.NiFiProperties; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.DisposableBean; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; +import org.xml.sax.SAXException; + +/** + * Factory bean for loading the configured authority provider. + */ +public class AuthorityProviderFactoryBean implements FactoryBean, ApplicationContextAware, DisposableBean, AuthorityProviderLookup { + + private static final Logger logger = LoggerFactory.getLogger(AuthorityProviderFactoryBean.class); + private static final String AUTHORITY_PROVIDERS_XSD = "/authority-providers.xsd"; + private static final String JAXB_GENERATED_PATH = "org.apache.nifi.authorization.generated"; + private static final JAXBContext JAXB_CONTEXT = initializeJaxbContext(); + + /** + * Load the JAXBContext. + */ + private static JAXBContext initializeJaxbContext() { + try { + return JAXBContext.newInstance(JAXB_GENERATED_PATH, AuthorityProviderFactoryBean.class.getClassLoader()); + } catch (JAXBException e) { + throw new RuntimeException("Unable to create JAXBContext."); + } + } + + private ApplicationContext applicationContext; + private AuthorityProvider authorityProvider; + private NiFiProperties properties; + private final Map authorityProviders = new HashMap<>(); + + @Override + public AuthorityProvider getAuthorityProvider(String identifier) { + return authorityProviders.get(identifier); + } + + @Override + public Object getObject() throws Exception { + if (authorityProvider == null) { + // look up the authority provider to use + final String authorityProviderIdentifier = properties.getProperty(NiFiProperties.SECURITY_USER_AUTHORITY_PROVIDER); + + // ensure the authority provider class name was specified + if (StringUtils.isBlank(authorityProviderIdentifier)) { + // if configured for ssl, the authority provider must be specified + if (properties.getSslPort() != null) { + throw new Exception("When running securely, the authority provider identifier must be specified in the nifi properties file."); + } + + // use a default provider... only allowable when running not securely + authorityProvider = createDefaultProvider(); + } else { + final AuthorityProviders authorityProviderConfiguration = loadAuthorityProvidersConfiguration(); + + // create each authority provider + for (final Provider provider : authorityProviderConfiguration.getProvider()) { + authorityProviders.put(provider.getIdentifier(), createAuthorityProvider(provider.getIdentifier(), provider.getClazz())); + } + + // configure each authority provider + for (final Provider provider : authorityProviderConfiguration.getProvider()) { + final AuthorityProvider instance = authorityProviders.get(provider.getIdentifier()); + instance.onConfigured(loadAuthorityProviderConfiguration(provider)); + } + + // get the authority provider instance + authorityProvider = getAuthorityProvider(authorityProviderIdentifier); + + // ensure it was found + if (authorityProvider == null) { + throw new Exception(String.format("The specified authority provider '%s' could not be found.", authorityProviderIdentifier)); + } + } + } + + return authorityProvider; + } + + /** + * Loads the authority providers configuration. + * + * @return + * @throws Exception + */ + private AuthorityProviders loadAuthorityProvidersConfiguration() throws Exception { + final File authorityProvidersConfigurationFile = properties.getAuthorityProviderConfiguraitonFile(); + + // load the users from the specified file + if (authorityProvidersConfigurationFile.exists()) { + try { + // find the schema + final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); + final Schema schema = schemaFactory.newSchema(AuthorityProviders.class.getResource(AUTHORITY_PROVIDERS_XSD)); + + // attempt to unmarshal + final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller(); + unmarshaller.setSchema(schema); + final JAXBElement element = unmarshaller.unmarshal(new StreamSource(authorityProvidersConfigurationFile), AuthorityProviders.class); + return element.getValue(); + } catch (SAXException | JAXBException e) { + throw new Exception("Unable to load the authority provider configuration file at: " + authorityProvidersConfigurationFile.getAbsolutePath()); + } + } else { + throw new Exception("Unable to find the authority provider configuration file at " + authorityProvidersConfigurationFile.getAbsolutePath()); + } + } + + /** + * Creates the AuthorityProvider instance for the identifier specified. + * + * @param identifier + * @param authorityProviderClassName + * @return + * @throws Exception + */ + private AuthorityProvider createAuthorityProvider(final String identifier, final String authorityProviderClassName) throws Exception { + // get the classloader for the specified authority provider + final ClassLoader authorityProviderClassLoader = ExtensionManager.getClassLoader(authorityProviderClassName); + if (authorityProviderClassLoader == null) { + throw new Exception(String.format("The specified authority provider class '%s' is not known to this nifi.", authorityProviderClassName)); + } + + // get the current context classloader + final ClassLoader currentClassLoader = Thread.currentThread().getContextClassLoader(); + + final AuthorityProvider instance; + try { + // set the appropriate class loader + Thread.currentThread().setContextClassLoader(authorityProviderClassLoader); + + // attempt to load the class + Class rawAuthorityProviderClass = Class.forName(authorityProviderClassName, true, authorityProviderClassLoader); + Class authorityProviderClass = rawAuthorityProviderClass.asSubclass(AuthorityProvider.class); + + // otherwise create a new instance + Constructor constructor = authorityProviderClass.getConstructor(); + instance = (AuthorityProvider) constructor.newInstance(); + + // method injection + performMethodInjection(instance, authorityProviderClass); + + // field injection + performFieldInjection(instance, authorityProviderClass); + + // call post construction lifecycle event + instance.initialize(new StandardAuthorityProviderInitializationContext(identifier, this)); + } finally { + if (currentClassLoader != null) { + Thread.currentThread().setContextClassLoader(currentClassLoader); + } + } + + return withNarLoader(instance); + } + + /** + * Loads the AuthorityProvider configuration. + * + * @param provider + * @return + */ + private AuthorityProviderConfigurationContext loadAuthorityProviderConfiguration(final Provider provider) { + final Map providerProperties = new HashMap<>(); + + for (final Property property : provider.getProperty()) { + providerProperties.put(property.getName(), property.getValue()); + } + + return new StandardAuthorityProviderConfigurationContext(provider.getIdentifier(), providerProperties); + } + + /** + * Performs method injection. + * + * @param instance + * @param authorityProviderClass + * @throws IllegalAccessException + * @throws IllegalArgumentException + * @throws InvocationTargetException + */ + private void performMethodInjection(final AuthorityProvider instance, final Class authorityProviderClass) throws IllegalAccessException, IllegalArgumentException, InvocationTargetException { + for (final Method method : authorityProviderClass.getMethods()) { + if (method.isAnnotationPresent(AuthorityProviderContext.class)) { + // make the method accessible + final boolean isAccessible = method.isAccessible(); + method.setAccessible(true); + + try { + final Class[] argumentTypes = method.getParameterTypes(); + + // look for setters (single argument) + if (argumentTypes.length == 1) { + final Class argumentType = argumentTypes[0]; + + // look for well known types + if (NiFiProperties.class.isAssignableFrom(argumentType)) { + // nifi properties injection + method.invoke(instance, properties); + } else if (ApplicationContext.class.isAssignableFrom(argumentType)) { + // spring application context injection + method.invoke(instance, applicationContext); + } + } + } finally { + method.setAccessible(isAccessible); + } + } + } + + final Class parentClass = authorityProviderClass.getSuperclass(); + if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) { + performMethodInjection(instance, parentClass); + } + } + + /** + * Performs field injection. + * + * @param instance + * @param authorityProviderClass + * @throws IllegalArgumentException + * @throws IllegalAccessException + */ + private void performFieldInjection(final AuthorityProvider instance, final Class authorityProviderClass) throws IllegalArgumentException, IllegalAccessException { + for (final Field field : authorityProviderClass.getDeclaredFields()) { + if (field.isAnnotationPresent(AuthorityProviderContext.class)) { + // make the method accessible + final boolean isAccessible = field.isAccessible(); + field.setAccessible(true); + + try { + // get the type + final Class fieldType = field.getType(); + + // only consider this field if it isn't set yet + if (field.get(instance) == null) { + // look for well known types + if (NiFiProperties.class.isAssignableFrom(fieldType)) { + // nifi properties injection + field.set(instance, properties); + } else if (ApplicationContext.class.isAssignableFrom(fieldType)) { + // spring application context injection + field.set(instance, applicationContext); + } + } + + } finally { + field.setAccessible(isAccessible); + } + } + } + + final Class parentClass = authorityProviderClass.getSuperclass(); + if (parentClass != null && AuthorityProvider.class.isAssignableFrom(parentClass)) { + performFieldInjection(instance, parentClass); + } + } + + /** + * Creates a default provider to use when running unsecurely with no + * provider configured. + * + * @return + */ + private AuthorityProvider createDefaultProvider() { + return new AuthorityProvider() { + @Override + public boolean doesDnExist(String dn) throws AuthorityAccessException { + return false; + } + + @Override + public Set getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException { + return EnumSet.noneOf(Authority.class); + } + + @Override + public void setAuthorities(String dn, Set authorities) throws UnknownIdentityException, AuthorityAccessException { + } + + @Override + public Set getUsers(Authority authority) throws AuthorityAccessException { + return new HashSet<>(); + } + + @Override + public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + } + + @Override + public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException { + } + + @Override + public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + return null; + } + + @Override + public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException { + } + + @Override + public void setUsersGroup(Set dn, String group) throws UnknownIdentityException, AuthorityAccessException { + } + + @Override + public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + } + + @Override + public void ungroup(String group) throws AuthorityAccessException { + } + + @Override + public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException { + } + + @Override + public void onConfigured(AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException { + } + + @Override + public void preDestruction() throws ProviderDestructionException { + } + }; + } + + /** + * Decorates the base provider to ensure the nar context classloader is used + * when invoking the underlying methods. + * + * @param baseProvider + * @return + */ + public AuthorityProvider withNarLoader(final AuthorityProvider baseProvider) { + return new AuthorityProvider() { + @Override + public boolean doesDnExist(String dn) throws AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return baseProvider.doesDnExist(dn); + } + } + + @Override + public Set getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return baseProvider.getAuthorities(dn); + } + } + + @Override + public void setAuthorities(String dn, Set authorities) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.setAuthorities(dn, authorities); + } + } + + @Override + public Set getUsers(Authority authority) throws AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return baseProvider.getUsers(authority); + } + } + + @Override + public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.revokeUser(dn); + } + } + + @Override + public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.addUser(dn, group); + } + } + + @Override + public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return baseProvider.getGroupForUser(dn); + } + } + + @Override + public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.revokeGroup(group); + } + } + + @Override + public void setUsersGroup(Set dns, String group) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.setUsersGroup(dns, group); + } + } + + @Override + public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.ungroupUser(dn); + } + } + + @Override + public void ungroup(String group) throws AuthorityAccessException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.ungroup(group); + } + } + + @Override + public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.initialize(initializationContext); + } + } + + @Override + public void onConfigured(AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.onConfigured(configurationContext); + } + } + + @Override + public void preDestruction() throws ProviderDestructionException { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + baseProvider.preDestruction(); + } + } + }; + } + + @Override + public Class getObjectType() { + return AuthorityProvider.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + @Override + public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { + this.applicationContext = applicationContext; + } + + @Override + public void destroy() throws Exception { + if (authorityProvider != null) { + authorityProvider.preDestruction(); + } + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java new file mode 100644 index 0000000000..0535e278df --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderConfigurationContext.java @@ -0,0 +1,50 @@ +/* + * 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.authorization; + +import java.util.Collections; +import java.util.Map; + +/** + * + */ +public class StandardAuthorityProviderConfigurationContext implements AuthorityProviderConfigurationContext { + + private final String identifier; + private final Map properties; + + public StandardAuthorityProviderConfigurationContext(String identifier, Map properties) { + this.identifier = identifier; + this.properties = properties; + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public Map getProperties() { + return Collections.unmodifiableMap(properties); + } + + @Override + public String getProperty(String property) { + return properties.get(property); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java new file mode 100644 index 0000000000..e4b16c4a94 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/authorization/StandardAuthorityProviderInitializationContext.java @@ -0,0 +1,42 @@ +/* + * 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.authorization; + +/** + * + */ +public class StandardAuthorityProviderInitializationContext implements AuthorityProviderInitializationContext { + + private final String identifier; + private final AuthorityProviderLookup authorityProviderLookup; + + public StandardAuthorityProviderInitializationContext(String identifier, AuthorityProviderLookup authorityProviderLookup) { + this.identifier = identifier; + this.authorityProviderLookup = authorityProviderLookup; + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public AuthorityProviderLookup getAuthorityProviderLookup() { + return authorityProviderLookup; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java new file mode 100644 index 0000000000..853687155a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/History.java @@ -0,0 +1,56 @@ +/* + * 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.history; + +import java.util.Collection; +import java.util.Date; +import org.apache.nifi.action.Action; + +/** + * The result of running an action query. + */ +public class History { + + private Integer total; + private Date lastRefreshed; + private Collection actions; + + public Collection getActions() { + return actions; + } + + public void setActions(Collection actions) { + this.actions = actions; + } + + public Integer getTotal() { + return total; + } + + public void setTotal(Integer totalRecordCount) { + this.total = totalRecordCount; + } + + public Date getLastRefreshed() { + return lastRefreshed; + } + + public void setLastRefreshed(Date lastRefreshed) { + this.lastRefreshed = lastRefreshed; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java new file mode 100644 index 0000000000..53cc13c1dd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/HistoryQuery.java @@ -0,0 +1,99 @@ +/* + * 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.history; + +import java.util.Date; + +/** + * + */ +public class HistoryQuery { + + private String userName; + private String sourceId; + private Date startDate; + private Date endDate; + private Integer offset; + private Integer count; + private String sortColumn; + private String sortOrder; + + public Date getEndDate() { + return endDate; + } + + public void setEndDate(Date endDate) { + this.endDate = endDate; + } + + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + public Date getStartDate() { + return startDate; + } + + public void setStartDate(Date startDate) { + this.startDate = startDate; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public Integer getOffset() { + return offset; + } + + public void setOffset(Integer offset) { + this.offset = offset; + } + + public Integer getCount() { + return count; + } + + public void setCount(Integer count) { + this.count = count; + } + + public String getSortColumn() { + return sortColumn; + } + + public void setSortColumn(String sortColumn) { + this.sortColumn = sortColumn; + } + + public String getSortOrder() { + return sortOrder; + } + + public void setSortOrder(String sortOrder) { + this.sortOrder = sortOrder; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java new file mode 100644 index 0000000000..6ece5cf127 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/history/PreviousValue.java @@ -0,0 +1,54 @@ +/* + * 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.history; + +import java.util.Date; + +/** + * + */ +public class PreviousValue { + + private String previousValue; + private Date timestamp; + private String userName; + + public String getPreviousValue() { + return previousValue; + } + + public void setPreviousValue(String previousValue) { + this.previousValue = previousValue; + } + + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java new file mode 100644 index 0000000000..195f6bf139 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/AccountStatus.java @@ -0,0 +1,47 @@ +/* + * 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.user; + +/** + * Represents the status of a user's account. + */ +public enum AccountStatus { + + ACTIVE, + PENDING, + DISABLED; + + /** + * Returns the matching status or null if the specified status does not + * match any statuses. + * + * @param rawStatus + * @return + */ + public static AccountStatus valueOfStatus(String rawStatus) { + AccountStatus desiredStatus = null; + + for (AccountStatus status : values()) { + if (status.toString().equals(rawStatus)) { + desiredStatus = status; + break; + } + } + + return desiredStatus; + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.java new file mode 100644 index 0000000000..984a572459 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUser.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.user; + +import java.io.Serializable; +import java.util.Date; +import java.util.EnumSet; +import java.util.Objects; +import java.util.Set; +import org.apache.nifi.authorization.Authority; +import org.apache.commons.lang3.StringUtils; + +/** + * An NiFiUser. + */ +public class NiFiUser implements Serializable { + + public static final String ANONYMOUS_USER_DN = "anonymous"; + + private String id; + private String dn; + private String userName; + private String userGroup; + private String justification; + + private Date creation; + private Date lastVerified; + private Date lastAccessed; + + private AccountStatus status; + private EnumSet authorities; + + /* getters / setters */ + public Date getCreation() { + return creation; + } + + public void setCreation(Date creation) { + this.creation = creation; + } + + public String getDn() { + return dn; + } + + public void setDn(String dn) { + this.dn = dn; + } + + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + public String getUserGroup() { + return userGroup; + } + + public void setUserGroup(String userGroup) { + this.userGroup = userGroup; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getJustification() { + return justification; + } + + public void setJustification(String justification) { + this.justification = justification; + } + + public AccountStatus getStatus() { + return status; + } + + public void setStatus(AccountStatus status) { + this.status = status; + } + + public Date getLastVerified() { + return lastVerified; + } + + public void setLastVerified(Date lastVerified) { + this.lastVerified = lastVerified; + } + + public Date getLastAccessed() { + return lastAccessed; + } + + public void setLastAccessed(Date lastAccessed) { + this.lastAccessed = lastAccessed; + } + + public Set getAuthorities() { + if (authorities == null) { + authorities = EnumSet.noneOf(Authority.class); + } + return authorities; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final NiFiUser other = (NiFiUser) obj; + if (!Objects.equals(this.dn, other.dn)) { + return false; + } + return true; + } + + @Override + public int hashCode() { + int hash = 7; + hash = 53 * hash + Objects.hashCode(this.dn); + return hash; + } + + @Override + public String toString() { + return String.format("dn[%s], userName[%s], justification[%s], authorities[%s]", getDn(), getUserName(), getJustification(), StringUtils.join(getAuthorities(), ", ")); + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java new file mode 100644 index 0000000000..7586fd13c0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/java/org/apache/nifi/user/NiFiUserGroup.java @@ -0,0 +1,45 @@ +/* + * 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.user; + +import java.util.Set; + +/** + * + */ +public class NiFiUserGroup { + + private String group; + private Set users; + + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + public Set getUsers() { + return users; + } + + public void setUsers(Set users) { + this.users = users; + } + +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml b/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml new file mode 100644 index 0000000000..a36619fa6f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/resources/nifi-administration-context.xml @@ -0,0 +1,62 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd b/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd new file mode 100644 index 0000000000..122fa2c32d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/main/xsd/authority-providers.xsd @@ -0,0 +1,49 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java new file mode 100644 index 0000000000..579770557c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/AuthorizeUserActionTest.java @@ -0,0 +1,433 @@ +/* + * 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.admin.service.action; + +import java.util.Date; +import java.util.EnumSet; +import java.util.Set; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountDisabledException; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AccountPendingException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * + */ +public class AuthorizeUserActionTest { + + private static final String USER_ID_6 = "6"; + private static final String USER_ID_7 = "7"; + private static final String USER_ID_8 = "8"; + private static final String USER_ID_9 = "9"; + private static final String USER_ID_10 = "10"; + private static final String USER_ID_11 = "11"; + + private static final String USER_DN_1 = "authority access exception while searching for user"; + private static final String USER_DN_2 = "unknown user"; + private static final String USER_DN_3 = "user removed after checking existence"; + private static final String USER_DN_4 = "access exception getting authorities"; + private static final String USER_DN_5 = "error creating user account"; + private static final String USER_DN_6 = "create user general sequence"; + private static final String USER_DN_7 = "existing user requires verification"; + private static final String USER_DN_8 = "existing user does not require verification"; + private static final String USER_DN_9 = "existing pending user"; + private static final String USER_DN_10 = "existing disabled user"; + private static final String USER_DN_11 = "existing user is now unknown in the authority provider"; + + private DAOFactory daoFactory; + private UserDAO userDao; + private AuthorityDAO authorityDao; + private AuthorityProvider authorityProvider; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String id = (String) args[0]; + + NiFiUser user = null; + if (USER_ID_7.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_7); + user.setDn(USER_DN_7); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + } else if (USER_ID_8.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_8); + user.setDn(USER_DN_8); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setLastVerified(new Date()); + } else if (USER_ID_11.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_11); + user.setDn(USER_DN_11); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setStatus(AccountStatus.ACTIVE); + } + + return user; + } + }).when(userDao).findUserById(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + NiFiUser user = null; + switch (dn) { + case USER_DN_7: + user = new NiFiUser(); + user.setId(USER_ID_7); + user.setDn(USER_DN_7); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + break; + case USER_DN_8: + user = new NiFiUser(); + user.setId(USER_ID_8); + user.setDn(USER_DN_8); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setLastVerified(new Date()); + break; + case USER_DN_9: + user = new NiFiUser(); + user.setId(USER_ID_9); + user.setDn(USER_DN_9); + user.setStatus(AccountStatus.PENDING); + break; + case USER_DN_10: + user = new NiFiUser(); + user.setId(USER_ID_10); + user.setDn(USER_DN_10); + user.setStatus(AccountStatus.DISABLED); + break; + case USER_DN_11: + user = new NiFiUser(); + user.setId(USER_ID_11); + user.setDn(USER_DN_11); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setStatus(AccountStatus.ACTIVE); + break; + } + + return user; + } + }).when(userDao).findUserByDn(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + switch (user.getDn()) { + case USER_DN_5: + throw new DataAccessException(); + case USER_DN_6: + user.setId(USER_ID_6); + break; + } + + // do nothing + return null; + } + }).when(userDao).createUser(Mockito.any(NiFiUser.class)); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + + // do nothing + return null; + } + }).when(userDao).updateUser(Mockito.any(NiFiUser.class)); + + // mock the authority dao + authorityDao = Mockito.mock(AuthorityDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Set authorities = (Set) args[0]; + String id = (String) args[1]; + + // do nothing + return null; + } + }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Set authorities = (Set) args[0]; + String id = (String) args[1]; + + // do nothing + return null; + } + }).when(authorityDao).deleteAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString()); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao); + + // mock the authority provider + authorityProvider = Mockito.mock(AuthorityProvider.class); + Mockito.doAnswer(new Answer() { + @Override + public Boolean answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + switch (dn) { + case USER_DN_1: + throw new AuthorityAccessException(StringUtils.EMPTY); + case USER_DN_2: + return false; + } + + return true; + } + }).when(authorityProvider).doesDnExist(Mockito.anyString()); + Mockito.doAnswer(new Answer>() { + @Override + public Set answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + Set authorities = EnumSet.noneOf(Authority.class); + switch (dn) { + case USER_DN_3: + throw new UnknownIdentityException(StringUtils.EMPTY); + case USER_DN_4: + throw new AuthorityAccessException(StringUtils.EMPTY); + case USER_DN_6: + authorities.add(Authority.ROLE_MONITOR); + break; + case USER_DN_7: + authorities.add(Authority.ROLE_DFM); + break; + case USER_DN_9: + throw new UnknownIdentityException(StringUtils.EMPTY); + case USER_DN_10: + throw new UnknownIdentityException(StringUtils.EMPTY); + case USER_DN_11: + throw new UnknownIdentityException(StringUtils.EMPTY); + } + + return authorities; + } + }).when(authorityProvider).getAuthorities(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + Set authorites = (Set) args[1]; + + // do nothing + return null; + } + }).when(authorityProvider).setAuthorities(Mockito.anyString(), Mockito.anySet()); + } + + /** + * Tests AuthorityAccessException in doesDnExist. + * + * @throws Exception + */ + @Test(expected = AdministrationException.class) + public void testAuthorityAccessExceptionInDoesDnExist() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_1, 0); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Test unknown user in the authority provider. + * + * @throws Exception + */ + @Test(expected = AccountNotFoundException.class) + public void testUnknownUser() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_2, 0); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Test a user thats been removed after checking their existence. + * + * @throws Exception + */ + @Test(expected = AccountNotFoundException.class) + public void testUserRemovedAfterCheckingExistence() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_3, 0); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Testing AuthorityAccessException when getting authorities. + * + * @throws Exception + */ + @Test(expected = AdministrationException.class) + public void testAuthorityAccessException() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_4, 0); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Testing DataAccessException while creating user accounts. + * + * @throws Exception + */ + @Test(expected = DataAccessException.class) + public void testErrorCreatingUserAccount() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_5, 0); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Tests the general case when a user account is created. + * + * @throws Exception + */ + @Test + public void testAccountCreation() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_6, 0); + NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider); + + // verify the user + Assert.assertEquals(USER_DN_6, user.getDn()); + Assert.assertEquals(1, user.getAuthorities().size()); + Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR)); + + // verify interaction with dao and provider + Mockito.verify(userDao, Mockito.times(1)).createUser(user); + } + + /** + * Tests the general case when there is an existing user account that + * requires verification. + * + * @throws Exception + */ + @Test + public void testExistingUserRequiresVerification() throws Exception { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_7, 0); + NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider); + + // verify the user + Assert.assertEquals(USER_DN_7, user.getDn()); + Assert.assertEquals(1, user.getAuthorities().size()); + Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM)); + + // verify interaction with dao and provider + Mockito.verify(userDao, Mockito.times(1)).updateUser(user); + Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_DFM), USER_ID_7); + } + + /** + * Tests the general case when there is an existing user account that does + * not require verification. + * + * @throws Exception + */ + @Test + public void testExistingUserNoVerification() throws Exception { + // disabling verification by passing in a large cache duration + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_8, Integer.MAX_VALUE); + NiFiUser user = authorizeUser.execute(daoFactory, authorityProvider); + + // verify the user + Assert.assertEquals(USER_DN_8, user.getDn()); + Assert.assertEquals(1, user.getAuthorities().size()); + Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR)); + + // verify interaction with dao and provider + Mockito.verify(userDao, Mockito.times(1)).updateUser(user); + Mockito.verify(authorityDao, Mockito.never()).createAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_8)); + Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_8)); + } + + /** + * Tests existing users whose accounts are in a pending status. + * + * @throws Exception + */ + @Test(expected = AccountPendingException.class) + public void testExistingPendingUser() throws Exception { + // disabling verification by passing in a large cache duration + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_9, Integer.MAX_VALUE); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Tests existing users whose accounts are in a disabled status. + * + * @throws Exception + */ + @Test(expected = AccountDisabledException.class) + public void testExistingDisabledUser() throws Exception { + // disabling verification by passing in a large cache duration + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_10, Integer.MAX_VALUE); + authorizeUser.execute(daoFactory, authorityProvider); + } + + /** + * Tests the general case where there is an active user that has been + * removed from the authority provider. + * + * @throws Exception + */ + @Test + public void testExistingActiveUserNotFoundInProvider() throws Exception { + try { + AuthorizeUserAction authorizeUser = new AuthorizeUserAction(USER_DN_11, 0); + authorizeUser.execute(daoFactory, authorityProvider); + + Assert.fail(); + } catch (AccountDisabledException ade) { + ArgumentCaptor user = ArgumentCaptor.forClass(NiFiUser.class); + + // verify interaction with dao + Mockito.verify(userDao, Mockito.times(1)).updateUser(user.capture()); + + // verify user + Assert.assertEquals(AccountStatus.DISABLED, user.getValue().getStatus()); + } + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java new file mode 100644 index 0000000000..3d2081b584 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/CreateUserActionTest.java @@ -0,0 +1,144 @@ +/* + * 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.admin.service.action; + +import java.util.EnumSet; +import java.util.Set; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * Test cases for creating a user. + */ +public class CreateUserActionTest { + + private String USER_ID_2 = "2"; + private String USER_ID_3 = "3"; + + private String USER_DN_1 = "data access exception when creating user"; + private String USER_DN_3 = "general create user case"; + + private DAOFactory daoFactory; + private UserDAO userDao; + private AuthorityDAO authorityDao; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + + if (USER_DN_1.equals(user.getDn())) { + throw new DataAccessException(); + } else if (USER_DN_3.equals(user.getDn())) { + user.setId(USER_ID_3); + } + + // do nothing + return null; + } + }).when(userDao).createUser(Mockito.any(NiFiUser.class)); + + // mock the authority dao + authorityDao = Mockito.mock(AuthorityDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Set authorities = (Set) args[0]; + String id = (String) args[1]; + + if (USER_ID_2.equals(id)) { + throw new DataAccessException(StringUtils.EMPTY); + } + + // do nothing + return null; + } + }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString()); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao); + } + + /** + * Tests DataAccessExceptions that occur while creating user accounts. + * + * @throws Exception + */ + @Test(expected = DataAccessException.class) + public void testExceptionCreatingUser() throws Exception { + NiFiUser user = new NiFiUser(); + user.setDn(USER_DN_1); + + CreateUserAction createUser = new CreateUserAction(user); + createUser.execute(daoFactory, null); + } + + /** + * Tests DataAccessExceptions that occur while create user authorities. + * + * @throws Exception + */ + @Test(expected = DataAccessException.class) + public void testExceptionCreatingAuthoroties() throws Exception { + NiFiUser user = new NiFiUser(); + user.setId(USER_ID_2); + + CreateUserAction createUser = new CreateUserAction(user); + createUser.execute(daoFactory, null); + } + + /** + * General case for creating a user. + * + * @throws Exception + */ + @Test + public void testCreateUserAccount() throws Exception { + NiFiUser user = new NiFiUser(); + user.setDn(USER_DN_3); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_DFM, Authority.ROLE_ADMIN)); + + CreateUserAction createUser = new CreateUserAction(user); + createUser.execute(daoFactory, null); + + // verify the user + Assert.assertEquals(USER_ID_3, user.getId()); + + // verify interaction with dao + Mockito.verify(userDao, Mockito.times(1)).createUser(user); + Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(user.getAuthorities(), USER_ID_3); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java new file mode 100644 index 0000000000..de852984ce --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/DisableUserActionTest.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * + */ +public class DisableUserActionTest { + + private static final String USER_ID_1 = "1"; + private static final String USER_ID_2 = "2"; + private static final String USER_ID_3 = "3"; + private static final String USER_ID_4 = "4"; + + private static final String USER_DN_3 = "authority access exception"; + private static final String USER_DN_4 = "general disable user case"; + + private DAOFactory daoFactory; + private UserDAO userDao; + private AuthorityProvider authorityProvider; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String id = (String) args[0]; + + NiFiUser user = null; + if (USER_ID_1.equals(id)) { + // leave user uninitialized + } else if (USER_ID_2.equals(id)) { + user = new NiFiUser(); + user.setId(id); + } else if (USER_ID_3.equals(id)) { + user = new NiFiUser(); + user.setId(id); + user.setDn(USER_DN_3); + } else if (USER_ID_4.equals(id)) { + user = new NiFiUser(); + user.setId(id); + user.setDn(USER_DN_4); + user.setStatus(AccountStatus.ACTIVE); + } + return user; + } + }).when(userDao).findUserById(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + + if (USER_ID_2.equals(user.getId())) { + throw new DataAccessException(StringUtils.EMPTY); + } + + // do nothing + return null; + } + }).when(userDao).updateUser(Mockito.any(NiFiUser.class)); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + + // mock the authority provider + authorityProvider = Mockito.mock(AuthorityProvider.class); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + if (USER_DN_3.equals(dn)) { + throw new AuthorityAccessException(StringUtils.EMPTY); + } + + // do nothing + return null; + } + }).when(authorityProvider).revokeUser(Mockito.anyString()); + } + + /** + * Tests the case when the user account is unknown. + * + * @throws Exception + */ + @Test(expected = AccountNotFoundException.class) + public void testUnknownUserAccount() throws Exception { + DisableUserAction disableUser = new DisableUserAction(USER_ID_1); + disableUser.execute(daoFactory, authorityProvider); + } + + /** + * Tests the case when a DataAccessException is thrown by the userDao. + * + * @throws Exception + */ + @Test(expected = DataAccessException.class) + public void testDataAccessExceptionInUserDao() throws Exception { + DisableUserAction disableUser = new DisableUserAction(USER_ID_2); + disableUser.execute(daoFactory, authorityProvider); + } + + /** + * Tests the case when a AuthorityAccessException is thrown by the provider. + * + * @throws Exception + */ + @Test(expected = AdministrationException.class) + public void testAuthorityAccessExceptionInProvider() throws Exception { + DisableUserAction disableUser = new DisableUserAction(USER_ID_3); + disableUser.execute(daoFactory, authorityProvider); + } + + /** + * Tests the general case when the user is disabled. + * + * @throws Exception + */ + @Test + public void testDisableUser() throws Exception { + DisableUserAction disableUser = new DisableUserAction(USER_ID_4); + NiFiUser user = disableUser.execute(daoFactory, authorityProvider); + + // verify the user + Assert.assertEquals(USER_ID_4, user.getId()); + Assert.assertEquals(USER_DN_4, user.getDn()); + Assert.assertEquals(AccountStatus.DISABLED, user.getStatus()); + + // verify the interaction with the dao and provider + Mockito.verify(userDao, Mockito.times(1)).updateUser(user); + Mockito.verify(authorityProvider, Mockito.times(1)).revokeUser(USER_DN_4); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java new file mode 100644 index 0000000000..93dbe619ca --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/InvalidateUserAccountActionTest.java @@ -0,0 +1,131 @@ +/* + * 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.admin.service.action; + +import java.util.Date; +import org.junit.Assert; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * Test case for InvalidateUserAccountAction. + */ +public class InvalidateUserAccountActionTest { + + private static final String USER_ID_1 = "1"; + private static final String USER_ID_2 = "2"; + private static final String USER_ID_3 = "3"; + + private DAOFactory daoFactory; + private UserDAO userDao; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String id = (String) args[0]; + + NiFiUser user = null; + if (USER_ID_1.equals(id)) { + // leave uninitialized + } else if (USER_ID_2.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_2); + } else if (USER_ID_3.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_3); + user.setLastVerified(new Date()); + } + return user; + } + }).when(userDao).findUserById(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + + if (USER_ID_2.equals(user.getId())) { + throw new DataAccessException(StringUtils.EMPTY); + } + + // do nothing + return null; + } + }).when(userDao).updateUser(Mockito.any(NiFiUser.class)); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + } + + /** + * Tests when the user account cannot be found. + * + * @throws Exception + */ + @Test(expected = AccountNotFoundException.class) + public void testAccountNotFoundException() throws Exception { + InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_1); + invalidateUserAccount.execute(daoFactory, null); + } + + /** + * Tests when a data access exception occurs when updating the user record. + * + * @throws Exception + */ + @Test(expected = DataAccessException.class) + public void testDataAccessException() throws Exception { + InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_2); + invalidateUserAccount.execute(daoFactory, null); + } + + /** + * Tests the general case of invalidating a user. + * + * @throws Exception + */ + @Test + public void testInvalidateUser() throws Exception { + InvalidateUserAccountAction invalidateUserAccount = new InvalidateUserAccountAction(USER_ID_3); + invalidateUserAccount.execute(daoFactory, null); + + // verify the interaction with the dao + ArgumentCaptor userCaptor = ArgumentCaptor.forClass(NiFiUser.class); + Mockito.verify(userDao, Mockito.times(1)).updateUser(userCaptor.capture()); + + // verify the user + NiFiUser user = userCaptor.getValue(); + Assert.assertEquals(USER_ID_3, user.getId()); + Assert.assertNull(user.getLastVerified()); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java new file mode 100644 index 0000000000..6e77d4657f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/RequestUserAccountActionTest.java @@ -0,0 +1,127 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.DataAccessException; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * Test case for RequestUserAccountAction. + */ +public class RequestUserAccountActionTest { + + private static final String USER_ID_3 = "3"; + + private static final String USER_DN_1 = "existing user account dn"; + private static final String USER_DN_2 = "data access exception"; + private static final String USER_DN_3 = "new account request"; + + private DAOFactory daoFactory; + private UserDAO userDao; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + NiFiUser user = null; + if (USER_DN_1.equals(dn)) { + user = new NiFiUser(); + } + return user; + } + }).when(userDao).findUserByDn(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + switch (user.getDn()) { + case USER_DN_2: + throw new DataAccessException(); + case USER_DN_3: + user.setId(USER_ID_3); + break; + } + + // do nothing + return null; + } + }).when(userDao).createUser(Mockito.any(NiFiUser.class)); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + } + + /** + * Tests when a user account already exists. + * + * @throws Exception + */ + @Test(expected = IllegalArgumentException.class) + public void testExistingAccount() throws Exception { + RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(USER_DN_1, StringUtils.EMPTY); + requestUserAccount.execute(daoFactory, null); + } + + /** + * Tests when a DataAccessException occurs while saving the new account + * request. + * + * @throws Exception + */ + @Test(expected = DataAccessException.class) + public void testDataAccessException() throws Exception { + RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(USER_DN_2, StringUtils.EMPTY); + requestUserAccount.execute(daoFactory, null); + } + + /** + * Tests the general case for requesting a new user account. + * + * @throws Exception + */ + @Test + public void testRequestUserAccountAction() throws Exception { + RequestUserAccountAction requestUserAccount = new RequestUserAccountAction(USER_DN_3, StringUtils.EMPTY); + NiFiUser user = requestUserAccount.execute(daoFactory, null); + + // verfiy the user + Assert.assertEquals(USER_ID_3, user.getId()); + Assert.assertEquals(USER_DN_3, user.getDn()); + Assert.assertEquals(AccountStatus.PENDING, user.getStatus()); + + // verify interaction with dao + Mockito.verify(userDao, Mockito.times(1)).createUser(user); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java new file mode 100644 index 0000000000..f37fc84edd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SeedUserAccountsActionTest.java @@ -0,0 +1,263 @@ +/* + * 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.admin.service.action; + +import org.apache.nifi.admin.service.action.SeedUserAccountsAction; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.Set; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.hamcrest.Matcher; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentMatcher; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * + */ +public class SeedUserAccountsActionTest { + + private static final String USER_ID_1 = "1"; + private static final String USER_ID_2 = "2"; + private static final String USER_ID_3 = "3"; + private static final String USER_ID_4 = "4"; + + private static final String USER_DN_1 = "user dn 1 - active user - remove monitor and operator, add dfm"; + private static final String USER_DN_2 = "user dn 2 - active user - no action"; + private static final String USER_DN_3 = "user dn 3 - pending user - add operator"; + private static final String USER_DN_4 = "user dn 4 - new user - add monitor"; + + private DAOFactory daoFactory; + private UserDAO userDao; + private AuthorityDAO authorityDao; + private AuthorityProvider authorityProvider; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String id = (String) args[0]; + + NiFiUser user = null; + if (USER_ID_1.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_1); + user.setDn(USER_DN_1); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setStatus(AccountStatus.ACTIVE); + } else if (USER_ID_2.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_2); + user.setDn(USER_DN_2); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN)); + user.setStatus(AccountStatus.ACTIVE); + } else if (USER_ID_3.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_3); + user.setDn(USER_DN_3); + user.setStatus(AccountStatus.PENDING); + } + return user; + } + }).when(userDao).findUserById(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + NiFiUser user = null; + if (USER_DN_1.equals(dn)) { + user = new NiFiUser(); + user.setId(USER_ID_1); + user.setDn(USER_DN_1); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setStatus(AccountStatus.ACTIVE); + } else if (USER_DN_2.equals(dn)) { + user = new NiFiUser(); + user.setId(USER_ID_2); + user.setDn(USER_DN_2); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN)); + user.setStatus(AccountStatus.ACTIVE); + } else if (USER_DN_3.equals(dn)) { + user = new NiFiUser(); + user.setId(USER_ID_3); + user.setDn(USER_DN_3); + user.setStatus(AccountStatus.PENDING); + } + return user; + } + }).when(userDao).findUserByDn(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + + if (USER_DN_4.equals(user.getDn())) { + user.setId(USER_ID_4); + } + + return null; + } + }).when(userDao).createUser(Mockito.any(NiFiUser.class)); + + // mock the authority dao + authorityDao = Mockito.mock(AuthorityDAO.class); + + // mock the authority provider + authorityProvider = Mockito.mock(AuthorityProvider.class); + Mockito.doAnswer(new Answer>() { + @Override + public Set answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Authority role = (Authority) args[0]; + + Set users = new HashSet<>(); + if (Authority.ROLE_DFM.equals(role)) { + users.add(USER_DN_1); + } else if (Authority.ROLE_ADMIN.equals(role)) { + users.add(USER_DN_2); + } else if (Authority.ROLE_PROXY.equals(role)) { + users.add(USER_DN_3); + } else if (Authority.ROLE_MONITOR.equals(role)) { + users.add(USER_DN_4); + } + return users; + } + }).when(authorityProvider).getUsers(Mockito.any(Authority.class)); + Mockito.doAnswer(new Answer>() { + @Override + public Set answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + Set authorities = EnumSet.noneOf(Authority.class); + switch (dn) { + case USER_DN_1: + authorities.add(Authority.ROLE_DFM); + break; + case USER_DN_2: + authorities.add(Authority.ROLE_ADMIN); + break; + case USER_DN_3: + authorities.add(Authority.ROLE_PROXY); + break; + case USER_DN_4: + authorities.add(Authority.ROLE_MONITOR); + break; + } + return authorities; + } + }).when(authorityProvider).getAuthorities(Mockito.anyString()); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao); + } + + /** + * Tests seeding the user accounts. + * + * @throws Exception + */ + @Test + public void testSeedUsers() throws Exception { + SeedUserAccountsAction seedUserAccounts = new SeedUserAccountsAction(); + seedUserAccounts.execute(daoFactory, authorityProvider); + + // matcher for user 1 + Matcher matchesUser1 = new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + NiFiUser user = (NiFiUser) argument; + return USER_ID_1.equals(user.getId()); + } + }; + + // verify user 1 - active existing user - remove monitor, operator, add dfm + Mockito.verify(userDao, Mockito.times(1)).updateUser(Mockito.argThat(matchesUser1)); + Mockito.verify(userDao, Mockito.never()).createUser(Mockito.argThat(matchesUser1)); + Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_DFM), USER_ID_1); + + // matcher for user 2 + Matcher matchesUser2 = new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + NiFiUser user = (NiFiUser) argument; + return USER_ID_2.equals(user.getId()); + } + }; + + // verify user 2 - active existing user - no actions + Mockito.verify(userDao, Mockito.times(1)).updateUser(Mockito.argThat(matchesUser2)); + Mockito.verify(userDao, Mockito.never()).createUser(Mockito.argThat(matchesUser2)); + Mockito.verify(authorityDao, Mockito.never()).createAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_2)); + Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_2)); + + // matchers for user 3 + Matcher matchesPendingUser3 = new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + NiFiUser user = (NiFiUser) argument; + return USER_ID_3.equals(user.getId()) && AccountStatus.ACTIVE.equals(user.getStatus()); + } + }; + Matcher matchesUser3 = new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + NiFiUser user = (NiFiUser) argument; + return USER_ID_3.equals(user.getId()); + } + }; + + // verify user 3 - pending user - add operator + Mockito.verify(userDao, Mockito.times(1)).updateUser(Mockito.argThat(matchesPendingUser3)); + Mockito.verify(userDao, Mockito.never()).createUser(Mockito.argThat(matchesUser3)); + Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_PROXY), USER_ID_3); + Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_3)); + + // matcher for user 4 + Matcher matchesUser4 = new ArgumentMatcher() { + @Override + public boolean matches(Object argument) { + NiFiUser user = (NiFiUser) argument; + return USER_ID_4.equals(user.getId()); + } + }; + + // verify user 4 - new user - add monitor + Mockito.verify(userDao, Mockito.never()).updateUser(Mockito.argThat(matchesUser4)); + Mockito.verify(userDao, Mockito.times(1)).createUser(Mockito.argThat(matchesUser4)); + Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_MONITOR), USER_ID_4); + Mockito.verify(authorityDao, Mockito.never()).deleteAuthorities(Mockito.anySet(), Mockito.eq(USER_ID_4)); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java new file mode 100644 index 0000000000..dd3695c0e5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/action/SetUserAuthoritiesActionTest.java @@ -0,0 +1,223 @@ +/* + * 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.admin.service.action; + +import java.util.Collections; +import java.util.EnumSet; +import java.util.Set; +import org.apache.nifi.admin.dao.AuthorityDAO; +import org.apache.nifi.admin.dao.DAOFactory; +import org.apache.nifi.admin.dao.UserDAO; +import org.apache.nifi.admin.service.AccountNotFoundException; +import org.apache.nifi.admin.service.AdministrationException; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.user.AccountStatus; +import org.apache.nifi.user.NiFiUser; +import org.apache.commons.lang3.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * Test case for SetUserAuthoritiesAction. + */ +public class SetUserAuthoritiesActionTest { + + private static final String USER_ID_1 = "1"; + private static final String USER_ID_2 = "2"; + private static final String USER_ID_3 = "3"; + + private static final String USER_DN_2 = "user dn 2"; + private static final String USER_DN_3 = "user dn 3"; + + private DAOFactory daoFactory; + private UserDAO userDao; + private AuthorityDAO authorityDao; + private AuthorityProvider authorityProvider; + + @Before + public void setup() throws Exception { + // mock the user dao + userDao = Mockito.mock(UserDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String id = (String) args[0]; + + NiFiUser user = null; + if (USER_ID_1.equals(id)) { + // leave user uninitialized + } else if (USER_ID_2.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_2); + user.setDn(USER_DN_2); + } else if (USER_ID_3.equals(id)) { + user = new NiFiUser(); + user.setId(USER_ID_3); + user.setDn(USER_DN_3); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setStatus(AccountStatus.ACTIVE); + } + return user; + } + }).when(userDao).findUserById(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public NiFiUser answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + NiFiUser user = null; + if (USER_DN_3.equals(dn)) { + user = new NiFiUser(); + user.setId(USER_ID_3); + user.setDn(USER_DN_3); + user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_MONITOR)); + user.setStatus(AccountStatus.ACTIVE); + } + return user; + } + }).when(userDao).findUserByDn(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + NiFiUser user = (NiFiUser) args[0]; + + // do nothing + return null; + } + }).when(userDao).updateUser(Mockito.any(NiFiUser.class)); + + // mock the authority dao + authorityDao = Mockito.mock(AuthorityDAO.class); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Set authorities = (Set) args[0]; + String id = (String) args[1]; + + // do nothing + return null; + } + }).when(authorityDao).createAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + Set authorities = (Set) args[0]; + String id = (String) args[1]; + + // do nothing + return null; + } + }).when(authorityDao).deleteAuthorities(Mockito.anySetOf(Authority.class), Mockito.anyString()); + + // mock the dao factory + daoFactory = Mockito.mock(DAOFactory.class); + Mockito.when(daoFactory.getUserDAO()).thenReturn(userDao); + Mockito.when(daoFactory.getAuthorityDAO()).thenReturn(authorityDao); + + // mock the authority provider + authorityProvider = Mockito.mock(AuthorityProvider.class); + Mockito.doAnswer(new Answer>() { + @Override + public Set answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + + Set authorities = EnumSet.noneOf(Authority.class); + if (USER_DN_3.equals(dn)) { + authorities.add(Authority.ROLE_DFM); + } + + return authorities; + } + }).when(authorityProvider).getAuthorities(Mockito.anyString()); + Mockito.doAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocation) throws Throwable { + Object[] args = invocation.getArguments(); + String dn = (String) args[0]; + Set authorites = (Set) args[1]; + + if (USER_DN_2.equals(dn)) { + throw new AuthorityAccessException(StringUtils.EMPTY); + } + + // do nothing + return null; + } + }).when(authorityProvider).setAuthorities(Mockito.anyString(), Mockito.anySet()); + } + + /** + * Test activating an unknown user account. User accounts are unknown then + * there is no pending account for the user. + * + * @throws Exception + */ + @Test(expected = AccountNotFoundException.class) + public void testUnknownUser() throws Exception { + UpdateUserAction setUserAuthorities = new UpdateUserAction(USER_ID_1, Collections.EMPTY_SET); + setUserAuthorities.execute(daoFactory, authorityProvider); + } + + /** + * Testing case then an AuthorityAccessException occurs while setting a + * users authorities. + * + * @throws Exception + */ + @Test(expected = AdministrationException.class) + public void testAuthorityAccessException() throws Exception { + UpdateUserAction setUserAuthorities = new UpdateUserAction(USER_ID_2, Collections.EMPTY_SET); + setUserAuthorities.execute(daoFactory, authorityProvider); + } + + /** + * Tests general case of setting user authorities. + * + * @throws Exception + */ + @Test + public void testSetAuthorities() throws Exception { + UpdateUserAction setUserAuthorities = new UpdateUserAction(USER_ID_3, EnumSet.of(Authority.ROLE_ADMIN)); + NiFiUser user = setUserAuthorities.execute(daoFactory, authorityProvider); + + // verify user + Assert.assertEquals(USER_ID_3, user.getId()); + Assert.assertEquals(1, user.getAuthorities().size()); + Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_ADMIN)); + + // verify interaction with dao + Mockito.verify(userDao, Mockito.times(1)).updateUser(user); + Mockito.verify(authorityDao, Mockito.times(1)).createAuthorities(EnumSet.of(Authority.ROLE_ADMIN), USER_ID_3); + + Set authoritiesAddedToProvider = EnumSet.of(Authority.ROLE_ADMIN); + + // verify interaction with provider + Mockito.verify(authorityProvider, Mockito.times(1)).setAuthorities(USER_DN_3, authoritiesAddedToProvider); + } +} diff --git a/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java new file mode 100644 index 0000000000..5c8b75ac73 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/administration/src/test/java/org/apache/nifi/admin/service/impl/NiFiAuthorizationServiceTest.java @@ -0,0 +1,284 @@ +/* + * 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.admin.service.impl; + +import org.junit.Ignore; + +/** + * + */ +@Ignore +public class NiFiAuthorizationServiceTest { + +// private static final String UNKNOWN_USER_IN_CACHE_DN = "unknown-user-in-cache-dn"; +// private static final String PENDING_USER_DN = "pending-user-dn"; +// private static final String DISABLED_USER_DN = "disabled-user-dn"; +// private static final String UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN = "unknown-user-in-identity-provider-dn"; +// private static final String ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN = "access-exception-in-identity-provider-dn"; +// private static final String UNABLE_TO_UPDATE_CACHE_DN = "unable-to-update-cache-dn"; +// private static final String VERIFICATION_REQUIRED_DN = "verification-required-dn"; +// private static final String VERIFICATION_NOT_REQUIRED_DN = "verification-not-required-dn"; +// private static final String NEW_USER_DN = "new-user-dn"; +// +// private UserService userService; +// private AuthorityProvider authorityProvider; +// private UserDAO userDAO; +// +// @Before +// public void setup() throws Exception { +// // mock the web security properties +// NiFiProperties properties = Mockito.mock(NiFiProperties.class); +// Mockito.when(properties.getSupportNewAccountRequests()).thenReturn(Boolean.TRUE); +// Mockito.when(properties.getUserCredentialCacheDurationSeconds()).thenReturn(60); +// +// // mock the authority provider +// +// // mock the admin service +// userDAO = Mockito.mock(UserDAO.class); +// Mockito.doAnswer(new Answer() { +// +// @Override +// public Object answer(InvocationOnMock invocation) throws Throwable { +// Object[] args = invocation.getArguments(); +// String dn = (String) args[0]; +// +// NiFiUser user = null; +// switch (dn) { +// case PENDING_USER_DN: +// user = new NiFiUser(); +// user.setDn(dn); +// user.setStatus(AccountStatus.PENDING); +// break; +// case DISABLED_USER_DN: +// user = new NiFiUser(); +// user.setDn(dn); +// user.setStatus(AccountStatus.DISABLED); +// break; +// case UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN: +// case UNABLE_TO_UPDATE_CACHE_DN: +// case ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN: +// user = new NiFiUser(); +// user.setDn(dn); +// user.setStatus(AccountStatus.ACTIVE); +// break; +// case VERIFICATION_REQUIRED_DN: { +// Calendar calendar = Calendar.getInstance(); +// calendar.add(Calendar.SECOND, -65); +// user = new NiFiUser(); +// user.setDn(dn); +// user.setStatus(AccountStatus.ACTIVE); +// user.setLastVerified(calendar.getTime()); +// user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN, Authority.ROLE_DFM)); +// break; +// } +// case VERIFICATION_NOT_REQUIRED_DN: { +// Calendar calendar = Calendar.getInstance(); +// calendar.add(Calendar.SECOND, -5); +// user = new NiFiUser(); +// user.setDn(dn); +// user.setStatus(AccountStatus.ACTIVE); +// user.setLastVerified(calendar.getTime()); +// user.getAuthorities().addAll(EnumSet.of(Authority.ROLE_ADMIN, Authority.ROLE_DFM)); +// break; +// } +// } +// return user; +// } +// }).when(userDAO).getUser(Mockito.anyString()); +// Mockito.doAnswer(new Answer() { +// +// @Override +// public Object answer(InvocationOnMock invocation) throws Throwable { +// Object[] args = invocation.getArguments(); +// NiFiUser user = (NiFiUser) args[0]; +// +// if (UNABLE_TO_UPDATE_CACHE_DN.equals(user.getDn())) { +// throw new AdministrationException(); +// } +// return user; +// } +// }).when(userDAO).updateUser(Mockito.any(NiFiUser.class)); +// Mockito.doNothing().when(userDAO).createUser(Mockito.any(NiFiUser.class)); +// +// // mock the authority provider +// authorityProvider = Mockito.mock(AuthorityProvider.class); +// Mockito.doAnswer(new Answer() { +// +// @Override +// public Object answer(InvocationOnMock invocation) throws Throwable { +// Object[] args = invocation.getArguments(); +// String dn = (String) args[0]; +// +// boolean hasDn = false; +// if (VERIFICATION_REQUIRED_DN.equals(dn) || NEW_USER_DN.equals(dn)) { +// hasDn = true; +// } +// return hasDn; +// } +// }).when(authorityProvider).doesDnExist(Mockito.anyString()); +// Mockito.doAnswer(new Answer() { +// +// @Override +// public Object answer(InvocationOnMock invocation) throws Throwable { +// Object[] args = invocation.getArguments(); +// String dn = (String) args[0]; +// +// Set authorities = null; +// switch (dn) { +// case VERIFICATION_REQUIRED_DN: +// case NEW_USER_DN: +// authorities = new HashSet<>(); +// authorities.add("ROLE_MONITOR"); +// break; +// case DISABLED_USER_DN: +// throw new UnknownIdentityException("Unable to find user"); +// } +// return authorities; +// } +// }).when(authorityProvider).getAuthorities(Mockito.anyString()); +// +// // create an instance of the authorization service +// userService = new UserServiceImpl(); +// ((UserServiceImpl) userService).setAuthorityProvider(authorityProvider); +// ((UserServiceImpl) userService).set(authorityProvider); +// +//// authorizationService.setIdentityProvider(identityProvider); +//// authorizationService.setAuthorityProvider(authorityProvider); +//// authorizationService.setProperties(properties); +// } +// +// /** +// * Ensures the authorization service correctly handles users who are +// * unknown. +// * +// * @throws Exception +// */ +// @Test(expected = org.springframework.security.core.userdetails.UsernameNotFoundException.class) +// public void testUnknownUserInCache() throws Exception { +// authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNKNOWN_USER_IN_CACHE_DN)); +// } +// +// /** +// * Ensures the authorization service correctly handles users whose accounts +// * are PENDING. +// * +// * @throws Exception +// */ +// @Test(expected = nifi.admin.service.AccountPendingException.class) +// public void testPendingUser() throws Exception { +// authorizationService.loadUserByUsername(WebUtils.formatProxyDn(PENDING_USER_DN)); +// } +// +// /** +// * Ensures the authorization service correctly handles users whose accounts +// * are DISABLED. +// * +// * @throws Exception +// */ +// @Test(expected = org.springframework.security.authentication.DisabledException.class) +// public void testDisabledUser() throws Exception { +// authorizationService.loadUserByUsername(WebUtils.formatProxyDn(DISABLED_USER_DN)); +// } +// +// /** +// * Ensures the authorization service correctly handles users whose are in +// * the cache but have been removed from the identity provider. +// * +// * @throws Exception +// */ +// @Test(expected = org.springframework.security.authentication.DisabledException.class) +// public void testUnknownUserInIdentityProvider() throws Exception { +// authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNKNOWN_USER_IN_IDENTITY_PROVIDER_DN)); +// } +// +// /** +// * Ensures the authorization service correctly handles cases when the cache +// * is unable to be updated. +// * +// * @throws Exception +// */ +// @Test(expected = org.springframework.security.authentication.AuthenticationServiceException.class) +// public void testUnableToUpdateCache() throws Exception { +// authorizationService.loadUserByUsername(WebUtils.formatProxyDn(UNABLE_TO_UPDATE_CACHE_DN)); +// } +// +// /** +// * Ensures the authorization service correctly handles cases when the +// * identity provider has an access exception. +// * +// * @throws Exception +// */ +// @Test(expected = org.springframework.security.authentication.AuthenticationServiceException.class) +// public void testUnableToAccessIdentity() throws Exception { +// authorizationService.loadUserByUsername(WebUtils.formatProxyDn(ACCESS_EXCEPTION_IN_IDENTITY_PROVIDER_DN)); +// } +// +// /** +// * Ensures that user authorities are properly loaded from the authority +// * provider. +// * +// * @throws Exception +// */ +// @Test +// public void testVerificationRequiredUser() throws Exception { +// NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(VERIFICATION_REQUIRED_DN)); +// NiFiUser user = userDetails.getNiFiUser(); +// Mockito.verify(authorityProvider).getAuthorities(VERIFICATION_REQUIRED_DN); +// +// // ensure the user details +// Assert.assertEquals(VERIFICATION_REQUIRED_DN, user.getDn()); +// Assert.assertEquals(1, user.getAuthorities().size()); +// Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR)); +// } +// +// /** +// * Ensures that user authorities are not loaded when the cache is still +// * valid. +// * +// * @throws Exception +// */ +// @Test +// public void testVerificationNotRequiredUser() throws Exception { +// NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(VERIFICATION_NOT_REQUIRED_DN)); +// NiFiUser user = userDetails.getNiFiUser(); +// Mockito.verify(authorityProvider, Mockito.never()).getAuthorities(VERIFICATION_NOT_REQUIRED_DN); +// +// // ensure the user details +// Assert.assertEquals(VERIFICATION_NOT_REQUIRED_DN, user.getDn()); +// Assert.assertEquals(2, user.getAuthorities().size()); +// Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_ADMIN)); +// Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_DFM)); +// } +// +// /** +// * Ensures that new users are automatically created when the authority +// * provider has their authorities. +// * +// * @throws Exception +// */ +// @Test +// public void testNewUser() throws Exception { +// NiFiUserDetails userDetails = (NiFiUserDetails) authorizationService.loadUserByUsername(WebUtils.formatProxyDn(NEW_USER_DN)); +// NiFiUser user = userDetails.getNiFiUser(); +// Mockito.verify(authorityProvider).getAuthorities(NEW_USER_DN); +// +// // ensure the user details +// Assert.assertEquals(NEW_USER_DN, user.getDn()); +// Assert.assertEquals(1, user.getAuthorities().size()); +// Assert.assertTrue(user.getAuthorities().contains(Authority.ROLE_MONITOR)); +// } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/.gitignore b/nar-bundles/framework-bundle/framework/client-dto/.gitignore new file mode 100755 index 0000000000..cd1a4e7077 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/.gitignore @@ -0,0 +1,6 @@ +/target +/target +/target +/target +/target +/target diff --git a/nar-bundles/framework-bundle/framework/client-dto/pom.xml b/nar-bundles/framework-bundle/framework/client-dto/pom.xml new file mode 100644 index 0000000000..4b210da0ba --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/pom.xml @@ -0,0 +1,47 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + + client-dto + NiFi Client Dto + + + + + org.apache.maven.plugins + maven-source-plugin + + + attach-sources + verify + + jar + + + + + + + diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.java new file mode 100644 index 0000000000..0e2dcb0c74 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/AboutDTO.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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Contains details about this NiFi including the title and version. + */ +@XmlType(name = "about") +public class AboutDTO { + + private String title; + private String version; + + /* getters / setters */ + /** + * The title to be used on the page and in the About dialog. + * + * @return The title + */ + public String getTitle() { + return title; + } + + public void setTitle(String title) { + this.title = title; + } + + /** + * The version of this NiFi. + * + * @return The version. + */ + public String getVersion() { + return version; + } + + public void setVersion(String version) { + this.version = version; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.java new file mode 100644 index 0000000000..70c408b0c7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BannerDTO.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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Banners that should appear on the top and bottom of this NiFi. + */ +@XmlType(name = "banners") +public class BannerDTO { + + private String headerText; + private String footerText; + + /* getters / setters */ + /** + * The banner footer text. + * + * @return The footer text + */ + public String getFooterText() { + return footerText; + } + + public void setFooterText(String footerText) { + this.footerText = footerText; + } + + /** + * The banner header text. + * + * @return The header text + */ + public String getHeaderText() { + return headerText; + } + + public void setHeaderText(String headerText) { + this.headerText = headerText; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java new file mode 100644 index 0000000000..ddc3d2e904 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinBoardDTO.java @@ -0,0 +1,63 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import java.util.List; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * The contents for the bulletin board including the bulletins and the timestamp + * when the board was generated. + */ +@XmlType(name = "bulletinBoard") +public class BulletinBoardDTO { + + private List bulletins; + private Date generated; + + /** + * The bulletins to populate in the bulletin board. + * + * @return + */ + public List getBulletins() { + return bulletins; + } + + public void setBulletins(List bulletins) { + this.bulletins = bulletins; + } + + /** + * When this bulletin board was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(final Date generated) { + this.generated = generated; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java new file mode 100644 index 0000000000..c6aca24051 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinDTO.java @@ -0,0 +1,161 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * A bulletin that represents a notification about a passing event including, + * the source component (if applicable), the timestamp, the message, and where + * the bulletin originated (if applicable). + */ +@XmlType(name = "bulletin") +public class BulletinDTO { + + private Long id; + private String nodeAddress; + private String category; + private String groupId; + private String sourceId; + private String sourceName; + private String level; + private String message; + private Date timestamp; + + /** + * The id of this message. + * + * @return + */ + public Long getId() { + return id; + } + + public void setId(Long id) { + this.id = id; + } + + /** + * When clustered, the address of the node from which this bulletin + * originated. + * + * @return + */ + public String getNodeAddress() { + return nodeAddress; + } + + public void setNodeAddress(String nodeAddress) { + this.nodeAddress = nodeAddress; + } + + /** + * The group id of the source component. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The category of this message. + * + * @return + */ + public String getCategory() { + return category; + } + + public void setCategory(String category) { + this.category = category; + } + + /** + * The actual message. + * + * @return + */ + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + /** + * The id of the source of this message. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The name of the source of this message. + * + * @return + */ + public String getSourceName() { + return sourceName; + } + + public void setSourceName(String sourceName) { + this.sourceName = sourceName; + } + + /** + * The level of this bulletin. + * + * @return + */ + public String getLevel() { + return level; + } + + public void setLevel(String level) { + this.level = level; + } + + /** + * When this bulletin was generated as a formatted string. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java new file mode 100644 index 0000000000..015b174bab --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/BulletinQueryDTO.java @@ -0,0 +1,113 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * A query for bulletin board. Will filter the resulting bulletin board + * according to the criteria in this query. + */ +@XmlType(name = "bulletinQuery") +public class BulletinQueryDTO { + + private String sourceId; + private String groupId; + private String name; + private String message; + private Long after; + private Integer limit; + + /** + * Include bulletins after this id. + * + * @return + */ + public Long getAfter() { + return after; + } + + public void setAfter(Long after) { + this.after = after; + } + + /** + * Include bulletin within this group. Supports a regular expression. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * Include bulletins that match this message. Supports a regular expression. + * + * @return + */ + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + /** + * Include bulletins that match this name. Supports a regular expression. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * Include bulletins that match this id. Supports a source id. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The maximum number of bulletins to return. + * + * @return + */ + public Integer getLimit() { + return limit; + } + + public void setLimit(Integer limit) { + this.limit = limit; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java new file mode 100644 index 0000000000..53100e3044 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ClusterDTO.java @@ -0,0 +1,61 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.Date; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * Details about the composition of the cluster at a specific date/time. + */ +@XmlType(name = "cluster") +public class ClusterDTO { + + private Collection nodes; + private Date generated; + + /** + * The collection of the node DTOs. + * + * @return + */ + public Collection getNodes() { + return nodes; + } + + public void setNodes(Collection nodes) { + this.nodes = nodes; + } + + /** + * Gets the date/time that this report was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java new file mode 100644 index 0000000000..1be480cfb1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectableDTO.java @@ -0,0 +1,146 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details about a connectable component. + */ +@XmlType(name = "connectable") +public class ConnectableDTO { + + private String id; + private String type; + private String groupId; + private String name; + private Boolean running; + private Boolean transmitting; + private Boolean exists; + private String comments; + + /** + * The id of this connectable component. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The type of this connectable component. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The id of the group that this connectable component resides in. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The name of this connectable component. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * Used to reflect the current state of this Connectable. + * + * @return + */ + public Boolean isRunning() { + return running; + } + + public void setRunning(Boolean running) { + this.running = running; + } + + /** + * If this represents a remote port it is used to indicate whether the + * target exists. + * + * @return + */ + public Boolean getExists() { + return exists; + } + + public void setExists(Boolean exists) { + this.exists = exists; + } + + /** + * If this represents a remote port it is used to indicate whether is it + * configured to transmit. + * + * @return + */ + public Boolean getTransmitting() { + return transmitting; + } + + public void setTransmitting(Boolean transmitting) { + this.transmitting = transmitting; + } + + /** + * The comments from this Connectable. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + @Override + public String toString() { + return "ConnectableDTO [Type=" + type + ", Name=" + name + ", Id=" + id + "]"; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java new file mode 100644 index 0000000000..660820c900 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ConnectionDTO.java @@ -0,0 +1,215 @@ +/* + * 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.web.api.dto; + +import java.util.List; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * A connection between two connectable components. + */ +@XmlType(name = "connection") +public class ConnectionDTO extends NiFiComponentDTO { + + private ConnectableDTO source; + private ConnectableDTO destination; + private String name; + private Integer labelIndex; + private Long zIndex; + private Set selectedRelationships; + private Set availableRelationships; + + private Long backPressureObjectThreshold; + private String backPressureDataSizeThreshold; + private String flowFileExpiration; + private List prioritizers; + private List bends; + + /** + * The id of the source processor. + * + * @return The id of the source processor + */ + public ConnectableDTO getSource() { + return source; + } + + public void setSource(ConnectableDTO source) { + this.source = source; + } + + /** + * The id of the target processor. + * + * @return The id of the target processor + */ + public ConnectableDTO getDestination() { + return destination; + } + + public void setDestination(ConnectableDTO destination) { + this.destination = destination; + } + + /** + * The name of the connection. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The position of the bend points on this connection. + * + * @return + */ + public List getBends() { + return bends; + } + + public void setBends(List bends) { + this.bends = bends; + } + + /** + * The index of control point that the connection label should be placed + * over. + * + * @return + */ + public Integer getLabelIndex() { + return labelIndex; + } + + public void setLabelIndex(Integer labelIndex) { + this.labelIndex = labelIndex; + } + + /** + * The z index for this connection. + * + * @return + */ + public Long getzIndex() { + return zIndex; + } + + public void setzIndex(Long zIndex) { + this.zIndex = zIndex; + } + + /** + * The relationships that make up this connection. + * + * @return The relationships + */ + public Set getSelectedRelationships() { + return selectedRelationships; + } + + public void setSelectedRelationships(Set relationships) { + this.selectedRelationships = relationships; + } + + /** + * The relationships that the source of the connection currently supports. + * This property is read only. + * + * @return + */ + public Set getAvailableRelationships() { + return availableRelationships; + } + + public void setAvailableRelationships(Set availableRelationships) { + this.availableRelationships = availableRelationships; + } + + /** + * The object count threshold for determining when back pressure is applied. + * Updating this value is a passive change in the sense that it won't impact + * whether existing files over the limit are affected but it does help + * feeder processors to stop pushing too much into this work queue. + * + * @return The back pressure object threshold + */ + public Long getBackPressureObjectThreshold() { + return backPressureObjectThreshold; + } + + public void setBackPressureObjectThreshold(Long backPressureObjectThreshold) { + this.backPressureObjectThreshold = backPressureObjectThreshold; + } + + /** + * The object data size threshold for determining when back pressure is + * applied. Updating this value is a passive change in the sense that it + * won't impact whether existing files over the limit are affected but it + * does help feeder processors to stop pushing too much into this work + * queue. + * + * @return The back pressure data size threshold + */ + public String getBackPressureDataSizeThreshold() { + return backPressureDataSizeThreshold; + } + + public void setBackPressureDataSizeThreshold(String backPressureDataSizeThreshold) { + this.backPressureDataSizeThreshold = backPressureDataSizeThreshold; + } + + /** + * The amount of time a flow file may be in the flow before it will be + * automatically aged out of the flow. Once a flow file reaches this age it + * will be terminated from the flow the next time a processor attempts to + * start work on it. + * + * @return The flow file expiration in minutes + */ + public String getFlowFileExpiration() { + return flowFileExpiration; + } + + public void setFlowFileExpiration(String flowFileExpiration) { + this.flowFileExpiration = flowFileExpiration; + } + + /** + * The prioritizers this processor is using. + * + * @return The prioritizer list + */ + public List getPrioritizers() { + return prioritizers; + } + + public void setPrioritizers(List prioritizers) { + this.prioritizers = prioritizers; + } + + @Override + public String toString() { + return "ConnectionDTO [name: " + name + " from " + source + " to " + destination + "]"; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java new file mode 100644 index 0000000000..b916025fd1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerConfigurationDTO.java @@ -0,0 +1,158 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details for the controller configuration. + */ +@XmlType(name = "config") +public class ControllerConfigurationDTO { + + private String name; + private String comments; + private Integer maxTimerDrivenThreadCount; + private Integer maxEventDrivenThreadCount; + + private Long autoRefreshIntervalSeconds; + private Boolean siteToSiteSecure; + + private Integer timeOffset; + + private String contentViewerUrl; + private String uri; + + /** + * The maximum number of timer driven threads this NiFi has available. + * + * @return The maximum number of threads + */ + public Integer getMaxTimerDrivenThreadCount() { + return maxTimerDrivenThreadCount; + } + + public void setMaxTimerDrivenThreadCount(Integer maxTimerDrivenThreadCount) { + this.maxTimerDrivenThreadCount = maxTimerDrivenThreadCount; + } + + /** + * The maximum number of event driven thread this NiFi has available. + * + * @return + */ + public Integer getMaxEventDrivenThreadCount() { + return maxEventDrivenThreadCount; + } + + public void setMaxEventDrivenThreadCount(Integer maxEventDrivenThreadCount) { + this.maxEventDrivenThreadCount = maxEventDrivenThreadCount; + } + + /** + * The name of this NiFi. + * + * @return The name + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The comments for this NiFi. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The interval in seconds between the automatic NiFi refresh requests. This + * value is read only. + * + * @return The interval in seconds + */ + public Long getAutoRefreshIntervalSeconds() { + return autoRefreshIntervalSeconds; + } + + public void setAutoRefreshIntervalSeconds(Long autoRefreshIntervalSeconds) { + this.autoRefreshIntervalSeconds = autoRefreshIntervalSeconds; + } + + /** + * Indicates whether or not Site-to-Site communications with this instance + * is secure (2-way authentication). This value is read only. + * + * @return + */ + public Boolean isSiteToSiteSecure() { + return siteToSiteSecure; + } + + public void setSiteToSiteSecure(Boolean siteToSiteSecure) { + this.siteToSiteSecure = siteToSiteSecure; + } + + /** + * The time offset of the server. + * + * @return + */ + public Integer getTimeOffset() { + return timeOffset; + } + + public void setTimeOffset(Integer timeOffset) { + this.timeOffset = timeOffset; + } + + /** + * Returns the URL for the content viewer if configured. + * + * @return + */ + public String getContentViewerUrl() { + return contentViewerUrl; + } + + public void setContentViewerUrl(String contentViewerUrl) { + this.contentViewerUrl = contentViewerUrl; + } + + /** + * The URI for this NiFi controller. + * + * @return + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java new file mode 100644 index 0000000000..9e15fc1772 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ControllerDTO.java @@ -0,0 +1,262 @@ +/* + * 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.web.api.dto; + +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * Configuration details for a NiFi controller. Primary use of this DTO is for + * consumption by a remote NiFi instance to initiate site to site + * communications. + */ +@XmlType(name = "controller") +public class ControllerDTO { + + private String id; + private String name; + private String comments; + + private Integer runningCount; + private Integer stoppedCount; + private Integer invalidCount; + private Integer disabledCount; + private Integer activeRemotePortCount; + private Integer inactiveRemotePortCount; + + private Integer inputPortCount; + private Integer outputPortCount; + + private Integer remoteSiteListeningPort; + private Boolean siteToSiteSecure; + private String instanceId; + private Set inputPorts; + private Set outputPorts; + + /** + * The id of this NiFi controller. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name of this NiFi controller. + * + * @return The name of this controller + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The comments of this NiFi controller. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The input ports available to send data to this NiFi controller. + * + * @return + */ + public Set getInputPorts() { + return inputPorts; + } + + public void setInputPorts(Set inputPorts) { + this.inputPorts = inputPorts; + } + + /** + * The output ports available to received data from this NiFi controller. + * + * @return + */ + public Set getOutputPorts() { + return outputPorts; + } + + public void setOutputPorts(Set outputPorts) { + this.outputPorts = outputPorts; + } + + /** + * The Instance ID of the cluster, if this node is connected to a Cluster + * Manager, or of this individual instance of in standalone mode + * + * @return + */ + public String getInstanceId() { + return instanceId; + } + + public void setInstanceId(String instanceId) { + this.instanceId = instanceId; + } + + /** + * The Socket Port on which this instance is listening for Remote Transfers + * of Flow Files. If this instance is not configured to receive Flow Files + * from remote instances, this will be null. + * + * @return a integer between 1 and 65535, or null, if not configured for + * remote transfer + */ + public Integer getRemoteSiteListeningPort() { + return remoteSiteListeningPort; + } + + public void setRemoteSiteListeningPort(final Integer port) { + this.remoteSiteListeningPort = port; + } + + /** + * Indicates whether or not Site-to-Site communications with this instance + * is secure (2-way authentication) + * + * @return + */ + public Boolean isSiteToSiteSecure() { + return siteToSiteSecure; + } + + public void setSiteToSiteSecure(Boolean siteToSiteSecure) { + this.siteToSiteSecure = siteToSiteSecure; + } + + /** + * The number of running components in this process group. + * + * @return + */ + public Integer getRunningCount() { + return runningCount; + } + + public void setRunningCount(Integer runningCount) { + this.runningCount = runningCount; + } + + /** + * The number of stopped components in this process group. + * + * @return + */ + public Integer getStoppedCount() { + return stoppedCount; + } + + public void setStoppedCount(Integer stoppedCount) { + this.stoppedCount = stoppedCount; + } + + /** + * The number of active remote ports contained in this process group. + * + * @return + */ + public Integer getActiveRemotePortCount() { + return activeRemotePortCount; + } + + public void setActiveRemotePortCount(Integer activeRemotePortCount) { + this.activeRemotePortCount = activeRemotePortCount; + } + + /** + * The number of inactive remote ports contained in this process group. + * + * @return + */ + public Integer getInactiveRemotePortCount() { + return inactiveRemotePortCount; + } + + public void setInactiveRemotePortCount(Integer inactiveRemotePortCount) { + this.inactiveRemotePortCount = inactiveRemotePortCount; + } + + /** + * The number of input ports contained in this process group. + * + * @return + */ + public Integer getInputPortCount() { + return inputPortCount; + } + + public void setInputPortCount(Integer inputPortCount) { + this.inputPortCount = inputPortCount; + } + + /** + * The number of invalid components in this process group. + * + * @return + */ + public Integer getInvalidCount() { + return invalidCount; + } + + public void setInvalidCount(Integer invalidCount) { + this.invalidCount = invalidCount; + } + + /** + * The number of disabled components in this process group. + * + * @return + */ + public Integer getDisabledCount() { + return disabledCount; + } + + public void setDisabledCount(Integer disabledCount) { + this.disabledCount = disabledCount; + } + + /** + * The number of output ports in this process group. + * + * @return + */ + public Integer getOutputPortCount() { + return outputPortCount; + } + + public void setOutputPortCount(Integer outputPortCount) { + this.outputPortCount = outputPortCount; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java new file mode 100644 index 0000000000..10ea41dc74 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CounterDTO.java @@ -0,0 +1,94 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Counter value for a specific component in a specific context. A counter is a + * value that a component can adjust during processing. + */ +@XmlType(name = "counter") +public class CounterDTO { + + private String id; + private String context; + private String name; + private Long valueCount; + private String value; + + /** + * The context of the counter. + * + * @return + */ + public String getContext() { + return context; + } + + public void setContext(String context) { + this.context = context; + } + + /** + * The id of the counter. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name of the counter + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The value for the counter + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + public Long getValueCount() { + return valueCount; + } + + public void setValueCount(Long valueCount) { + this.valueCount = valueCount; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java new file mode 100644 index 0000000000..ac1aa388df --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/CountersDTO.java @@ -0,0 +1,61 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.Date; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * All the counters in this NiFi instance at a given time. + */ +@XmlType(name = "counters") +public class CountersDTO { + + private Date generated; + private Collection counters; + + /** + * Gets the collection of counters. + * + * @return + */ + public Collection getCounters() { + return counters; + } + + public void setCounters(Collection counters) { + this.counters = counters; + } + + /** + * Gets the date/time that this report was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java new file mode 100644 index 0000000000..5a2d78905c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/DocumentedTypeDTO.java @@ -0,0 +1,71 @@ +/* + * 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.web.api.dto; + +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * Class used for providing documentation of a specified type that may be + * instantiated. + */ +@XmlType(name = "documentedType") +public class DocumentedTypeDTO { + + private String type; + private String description; + private Set tags; + + /** + * An optional description of the corresponding type. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * The type is the fully-qualified name of a Java class. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The tags associated with this type + * + * @return + */ + public Set getTags() { + return tags; + } + + public void setTags(final Set tags) { + this.tags = tags; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java new file mode 100644 index 0000000000..61c3c33762 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FlowSnippetDTO.java @@ -0,0 +1,141 @@ +/* + * 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.web.api.dto; + +import java.util.LinkedHashSet; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * The contents of a flow snippet. + */ +@XmlType(name = "flowSnippet") +public class FlowSnippetDTO { + + private Set processGroups = new LinkedHashSet<>(); + private Set remoteProcessGroups = new LinkedHashSet<>(); + private Set processors = new LinkedHashSet<>(); + private Set inputPorts = new LinkedHashSet<>(); + private Set outputPorts = new LinkedHashSet<>(); + private Set connections = new LinkedHashSet<>(); + private Set labels = new LinkedHashSet<>(); + private Set funnels = new LinkedHashSet<>(); + + /** + * The connections in this flow snippet. + * + * @return + */ + public Set getConnections() { + return connections; + } + + public void setConnections(Set connections) { + this.connections = connections; + } + + /** + * The input ports in this flow snippet. + * + * @return + */ + public Set getInputPorts() { + return inputPorts; + } + + public void setInputPorts(Set inputPorts) { + this.inputPorts = inputPorts; + } + + /** + * The labels in this flow snippet. + * + * @return + */ + public Set getLabels() { + return labels; + } + + public void setLabels(Set labels) { + this.labels = labels; + } + + /** + * The funnels in this flow snippet. + * + * @return + */ + public Set getFunnels() { + return funnels; + } + + public void setFunnels(Set funnels) { + this.funnels = funnels; + } + + /** + * The output ports in this flow snippet. + * + * @return + */ + public Set getOutputPorts() { + return outputPorts; + } + + public void setOutputPorts(Set outputPorts) { + this.outputPorts = outputPorts; + } + + /** + * The process groups in this flow snippet. + * + * @return + */ + public Set getProcessGroups() { + return processGroups; + } + + public void setProcessGroups(Set processGroups) { + this.processGroups = processGroups; + } + + /** + * The processors in this flow group. + * + * @return + */ + public Set getProcessors() { + return processors; + } + + public void setProcessors(Set processors) { + this.processors = processors; + } + + /** + * The remote process groups in this flow snippet. + * + * @return + */ + public Set getRemoteProcessGroups() { + return remoteProcessGroups; + } + + public void setRemoteProcessGroups(Set remoteProcessGroups) { + this.remoteProcessGroups = remoteProcessGroups; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java new file mode 100644 index 0000000000..1240501693 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/FunnelDTO.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details of a funnel. + */ +@XmlType(name = "funnel") +public class FunnelDTO extends NiFiComponentDTO { + + public FunnelDTO() { + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java new file mode 100644 index 0000000000..f50c79292b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/LabelDTO.java @@ -0,0 +1,94 @@ +/* + * 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.web.api.dto; + +import java.util.Collections; +import java.util.Map; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details of a label. + */ +@XmlType(name = "label") +public class LabelDTO extends NiFiComponentDTO { + + private String label; + + private Double width; + private Double height; + + // font-size = 12px + // color = #eee + private Map style = Collections.emptyMap(); + + public LabelDTO() { + } + + /** + * The text that appears in the label. + * + * @return The label text + */ + public String getLabel() { + return label; + } + + public void setLabel(final String label) { + this.label = label; + } + + /** + * The style for this label. + * + * @return + */ + public Map getStyle() { + return style; + } + + public void setStyle(Map style) { + this.style = style; + } + + /** + * The height of the label in pixels when at a 1:1 scale. + * + * @return + */ + public Double getHeight() { + return height; + } + + public void setHeight(Double height) { + this.height = height; + } + + /** + * The width of the label in pixels when at a 1:1 scale. + * + * @return + */ + public Double getWidth() { + return width; + } + + public void setWidth(Double width) { + this.width = width; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java new file mode 100644 index 0000000000..e3c844555b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NiFiComponentDTO.java @@ -0,0 +1,95 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Base class for all nifi components. + */ +@XmlType(name = "nifiComponent") +public class NiFiComponentDTO { + + private String id; + private String uri; + private PositionDTO position; + private String parentGroupId; + + public NiFiComponentDTO() { + } + + public NiFiComponentDTO(final String id) { + this.id = id; + } + + public NiFiComponentDTO(final String id, final double x, final double y) { + this.id = id; + this.position = new PositionDTO(x, y); + } + + /** + * The id for this component. + * + * @return The id + */ + public String getId() { + return this.id; + } + + public void setId(final String id) { + this.id = id; + } + + /** + * The id for the parent group of this component. + * + * @return + */ + public String getParentGroupId() { + return parentGroupId; + } + + public void setParentGroupId(String parentGroupId) { + this.parentGroupId = parentGroupId; + } + + /** + * The uri for linking to this component in this NiFi. + * + * @return The uri + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + public void setPosition(final PositionDTO position) { + this.position = position; + } + + /** + * The position of this component in the UI. + * + * @return The position + */ + public PositionDTO getPosition() { + return position; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java new file mode 100644 index 0000000000..9499c2efb2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeDTO.java @@ -0,0 +1,188 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import java.util.List; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * Details of a node within this NiFi. + */ +@XmlType(name = "node") +public class NodeDTO { + + private String nodeId; + private String address; + private Integer apiPort; + private String status; + private Date heartbeat; + private Date connectionRequested; + private Boolean primary; + private Integer activeThreadCount; + private String queued; + private List events; + private Date nodeStartTime; + + /** + * The node's last heartbeat timestamp. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getHeartbeat() { + return heartbeat; + } + + public void setHeartbeat(Date heartbeat) { + this.heartbeat = heartbeat; + } + + /** + * The time of the node's last connection request. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getConnectionRequested() { + return connectionRequested; + } + + public void setConnectionRequested(Date connectionRequested) { + this.connectionRequested = connectionRequested; + } + + /** + * The active thread count. + * + * @return The active thread count + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * The queue for the controller. + * + * @return + */ + public String getQueued() { + return queued; + } + + public void setQueued(String queued) { + this.queued = queued; + } + + /** + * The node's host/IP address. + * + * @return + */ + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + /** + * The node ID. + * + * @return + */ + public String getNodeId() { + return nodeId; + } + + public void setNodeId(String nodeId) { + this.nodeId = nodeId; + } + + /** + * The port the node is listening for API requests. + * + * @return + */ + public Integer getApiPort() { + return apiPort; + } + + public void setApiPort(Integer port) { + this.apiPort = port; + } + + /** + * The node's status. + * + * @return + */ + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + /** + * The node's events. + * + * @return + */ + public List getEvents() { + return events; + } + + public void setEvents(List events) { + this.events = events; + } + + /** + * Whether this node is the primary node within the cluster. + * + * @return + */ + public Boolean isPrimary() { + return primary; + } + + public void setPrimary(Boolean primary) { + this.primary = primary; + } + + /** + * The time at which this Node was last restarted + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getNodeStartTime() { + return nodeStartTime; + } + + public void setNodeStartTime(Date nodeStartTime) { + this.nodeStartTime = nodeStartTime; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java new file mode 100644 index 0000000000..3cad8d84e4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeEventDTO.java @@ -0,0 +1,74 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * A event for a node within this NiFi cluster. + */ +@XmlType(name = "nodeEvent") +public class NodeEventDTO { + + private Date timestamp; + private String category; + private String message; + + /** + * The category of the node event. + * + * @return + */ + public String getCategory() { + return category; + } + + public void setCategory(String category) { + this.category = category; + } + + /** + * The message of the node event. + * + * @return + */ + public String getMessage() { + return message; + } + + public void setMessage(String message) { + this.message = message; + } + + /** + * The timestamp of the node event. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java new file mode 100644 index 0000000000..8c83331005 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/NodeSystemDiagnosticsDTO.java @@ -0,0 +1,56 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * The system diagnostics for a node with this NiFi cluster. + */ +@XmlType(name = "nodeSystemDiagnostics") +public class NodeSystemDiagnosticsDTO { + + private NodeDTO node; + private SystemDiagnosticsDTO systemDiagnostics; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The system diagnostics. + * + * @return + */ + public SystemDiagnosticsDTO getSystemDiagnostics() { + return systemDiagnostics; + } + + public void setControllerStatus(SystemDiagnosticsDTO systemDiagnostics) { + this.systemDiagnostics = systemDiagnostics; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java new file mode 100644 index 0000000000..2a372f4ef4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PortDTO.java @@ -0,0 +1,161 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * The details for a port within this NiFi flow. + */ +@XmlType(name = "port") +public class PortDTO extends NiFiComponentDTO { + + private String name; + private String comments; + private String state; + private String type; + private Boolean transmitting; + private Integer concurrentlySchedulableTaskCount; + private Set userAccessControl; + private Set groupAccessControl; + + private Collection validationErrors; + + /** + * The name of this port. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(final String name) { + this.name = name; + } + + /** + * The state of this port. Possible states are 'RUNNING', 'STOPPED', and + * 'DISABLED'. + * + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The type of port. Possible values are 'INPUT_PORT' or 'OUTPUT_PORT'. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The number of tasks that should be concurrently scheduled for this port. + * + * @return + */ + public Integer getConcurrentlySchedulableTaskCount() { + return concurrentlySchedulableTaskCount; + } + + public void setConcurrentlySchedulableTaskCount(Integer concurrentlySchedulableTaskCount) { + this.concurrentlySchedulableTaskCount = concurrentlySchedulableTaskCount; + } + + /** + * The comments for this port. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * Whether this port has incoming or outgoing connections to a remote NiFi. + * This is only applicable when the port is running on the root group. + * + * @return + */ + public Boolean isTransmitting() { + return transmitting; + } + + public void setTransmitting(Boolean transmitting) { + this.transmitting = transmitting; + } + + /** + * Groups that are allowed to access this port. + * + * @return + */ + public Set getGroupAccessControl() { + return groupAccessControl; + } + + public void setGroupAccessControl(Set groupAccessControl) { + this.groupAccessControl = groupAccessControl; + } + + /** + * Users that are allowed to access this port. + * + * @return + */ + public Set getUserAccessControl() { + return userAccessControl; + } + + public void setUserAccessControl(Set userAccessControl) { + this.userAccessControl = userAccessControl; + } + + /** + * Gets the validation errors from this port. These validation errors + * represent the problems with the port that must be resolved before it can + * be started. + * + * @return The validation errors + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java new file mode 100644 index 0000000000..ab077f371b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PositionDTO.java @@ -0,0 +1,65 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * A position on the canvas. + */ +@XmlType(name = "position") +public class PositionDTO { + + private Double x; + private Double y; + + public PositionDTO() { + } + + public PositionDTO(Double x, Double y) { + this.x = x; + this.y = y; + } + + /* getters / setters */ + /** + * The x coordinate. + * + * @return + */ + public Double getX() { + return x; + } + + public void setX(Double x) { + this.x = x; + } + + /** + * The y coordinate. + * + * @return + */ + public Double getY() { + return y; + } + + public void setY(Double y) { + this.y = y; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java new file mode 100644 index 0000000000..fb33c672d3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PreviousValueDTO.java @@ -0,0 +1,73 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * The previous value for a processor property. + */ +@XmlType(name = "previousValue") +public class PreviousValueDTO { + + private String previousValue; + private Date timestamp; + private String userName; + + /** + * The previous value. + * + * @return + */ + public String getPreviousValue() { + return previousValue; + } + + public void setPreviousValue(String previousValue) { + this.previousValue = previousValue; + } + + /** + * When it was modified. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + /** + * The user who changed the previous value. + * + * @return + */ + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java new file mode 100644 index 0000000000..414004601f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessGroupDTO.java @@ -0,0 +1,219 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * The details for a process group within this NiFi flow. + */ +@XmlType(name = "processGroup") +public class ProcessGroupDTO extends NiFiComponentDTO { + + private String name; + private String comments; + private Boolean running; + + private ProcessGroupDTO parent; + + private Integer runningCount; + private Integer stoppedCount; + private Integer invalidCount; + private Integer disabledCount; + private Integer activeRemotePortCount; + private Integer inactiveRemotePortCount; + + private Integer inputPortCount; + private Integer outputPortCount; + + private FlowSnippetDTO contents; + + public ProcessGroupDTO() { + super(); + } + + /** + * The name of this Process Group. + * + * @return The name of this Process Group + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * This Process Group's parent + * + * @return This Process Group's parent + */ + public ProcessGroupDTO getParent() { + return parent; + } + + public void setParent(ProcessGroupDTO parent) { + this.parent = parent; + } + + /** + * The comments for this process group. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The contents of this process group. This field will be populated if the + * request is marked verbose. + * + * @return + */ + public FlowSnippetDTO getContents() { + return contents; + } + + public void setContents(FlowSnippetDTO contents) { + this.contents = contents; + } + + /** + * The number of input ports contained in this process group. + * + * @return + */ + public Integer getInputPortCount() { + return inputPortCount; + } + + public void setInputPortCount(Integer inputPortCount) { + this.inputPortCount = inputPortCount; + } + + /** + * The number of invalid components in this process group. + * + * @return + */ + public Integer getInvalidCount() { + return invalidCount; + } + + public void setInvalidCount(Integer invalidCount) { + this.invalidCount = invalidCount; + } + + /** + * The number of output ports in this process group. + * + * @return + */ + public Integer getOutputPortCount() { + return outputPortCount; + } + + public void setOutputPortCount(Integer outputPortCount) { + this.outputPortCount = outputPortCount; + } + + /** + * Used in requests, indicates whether this process group should be running. + * + * @return + */ + public Boolean isRunning() { + return running; + } + + public void setRunning(Boolean running) { + this.running = running; + } + + /** + * The number of running component in this process group. + * + * @return + */ + public Integer getRunningCount() { + return runningCount; + } + + public void setRunningCount(Integer runningCount) { + this.runningCount = runningCount; + } + + /** + * The number of stopped components in this process group. + * + * @return + */ + public Integer getStoppedCount() { + return stoppedCount; + } + + public void setStoppedCount(Integer stoppedCount) { + this.stoppedCount = stoppedCount; + } + + /** + * The number of disabled components in this process group. + * + * @return + */ + public Integer getDisabledCount() { + return disabledCount; + } + + public void setDisabledCount(Integer disabledCount) { + this.disabledCount = disabledCount; + } + + /** + * The number of active remote ports in this process group. + * + * @return + */ + public Integer getActiveRemotePortCount() { + return activeRemotePortCount; + } + + public void setActiveRemotePortCount(Integer activeRemotePortCount) { + this.activeRemotePortCount = activeRemotePortCount; + } + + /** + * The number of inactive remote ports in this process group. + * + * @return + */ + public Integer getInactiveRemotePortCount() { + return inactiveRemotePortCount; + } + + public void setInactiveRemotePortCount(Integer inactiveRemotePortCount) { + this.inactiveRemotePortCount = inactiveRemotePortCount; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java new file mode 100644 index 0000000000..1481b0f9d7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorConfigDTO.java @@ -0,0 +1,486 @@ +/* + * 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.web.api.dto; + +import java.util.Map; +import java.util.Set; + +import javax.xml.bind.annotation.XmlType; + +/** + * Configuration details for a processor in this NiFi. + */ +@XmlType(name = "processorConfig") +public class ProcessorConfigDTO { + + private Map properties; + private Map descriptors; + + // settings + private String schedulingPeriod; + private String schedulingStrategy; + private String penaltyDuration; + private String yieldDuration; + private String bulletinLevel; + private Long runDurationMillis; + private Integer concurrentlySchedulableTaskCount; + private Set autoTerminatedRelationships; + private String comments; + private String customUiUrl; + private Boolean lossTolerant; + + // annotation data + private String annotationData; + + private Map defaultConcurrentTasks; + private Map defaultSchedulingPeriod; + + public ProcessorConfigDTO() { + + } + + /** + * The amount of time that should elapse between task executions. This will + * not affect currently scheduled tasks. + * + * @return The scheduling period in seconds + */ + public String getSchedulingPeriod() { + return schedulingPeriod; + } + + public void setSchedulingPeriod(String setSchedulingPeriod) { + this.schedulingPeriod = setSchedulingPeriod; + } + + /** + * Indicates whether the processor should be scheduled to run in + * event-driven mode or timer-driven mode + * + * @return + */ + public String getSchedulingStrategy() { + return schedulingStrategy; + } + + public void setSchedulingStrategy(String schedulingStrategy) { + this.schedulingStrategy = schedulingStrategy; + } + + /** + * The amount of time that is used when this processor penalizes a flow + * file. + * + * @return + */ + public String getPenaltyDuration() { + return penaltyDuration; + } + + public void setPenaltyDuration(String penaltyDuration) { + this.penaltyDuration = penaltyDuration; + } + + /** + * When yielding, this amount of time must elaspe before this processor is + * scheduled again. + * + * @return + */ + public String getYieldDuration() { + return yieldDuration; + } + + public void setYieldDuration(String yieldDuration) { + this.yieldDuration = yieldDuration; + } + + /** + * The level at this this processor will report bulletins. + * + * @return + */ + public String getBulletinLevel() { + return bulletinLevel; + } + + public void setBulletinLevel(String bulletinLevel) { + this.bulletinLevel = bulletinLevel; + } + + /** + * The number of tasks that should be concurrently scheduled for this + * processor. If this processor doesn't allow parallel processing then any + * positive input will be ignored. + * + * @return The concurrently schedulable task count + */ + public Integer getConcurrentlySchedulableTaskCount() { + return concurrentlySchedulableTaskCount; + } + + public void setConcurrentlySchedulableTaskCount(Integer concurrentlySchedulableTaskCount) { + this.concurrentlySchedulableTaskCount = concurrentlySchedulableTaskCount; + } + + /** + * Whether or not this Processor is Loss Tolerant + * + * @return + */ + public Boolean isLossTolerant() { + return lossTolerant; + } + + public void setLossTolerant(final Boolean lossTolerant) { + this.lossTolerant = lossTolerant; + } + + /** + * The comments for this processor. + * + * @return The comments + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The properties for this processor. Properties whose value is not set will + * only contain the property name. These properties are (un)marshalled + * differently since we need/want to control the ordering of the properties. + * The descriptors and metadata are used as a lookup when processing these + * properties. + * + * @return The optional properties + */ + public Map getProperties() { + return properties; + } + + public void setProperties(Map properties) { + this.properties = properties; + } + + /** + * The descriptors for this processor's properties. + * + * @return + */ + public Map getDescriptors() { + return descriptors; + } + + public void setDescriptors(Map descriptors) { + this.descriptors = descriptors; + } + + /** + * Annotation data for this processor. + * + * @return The annotation data + */ + public String getAnnotationData() { + return annotationData; + } + + public void setAnnotationData(String annotationData) { + this.annotationData = annotationData; + } + + /** + * Whether of not this processor has a custom UI. + * + * @return + */ + public String getCustomUiUrl() { + return customUiUrl; + } + + public void setCustomUiUrl(String customUiUrl) { + this.customUiUrl = customUiUrl; + } + + /** + * The names of all processor relationships that cause a flow file to be + * terminated if the relationship is not connected to anything + * + * @return + */ + public Set getAutoTerminatedRelationships() { + return autoTerminatedRelationships; + } + + public void setAutoTerminatedRelationships(final Set autoTerminatedRelationships) { + this.autoTerminatedRelationships = autoTerminatedRelationships; + } + + /** + * Maps default values for concurrent tasks for each applicable scheduling + * strategy. + * + * @return + */ + public Map getDefaultConcurrentTasks() { + return defaultConcurrentTasks; + } + + public void setDefaultConcurrentTasks(Map defaultConcurrentTasks) { + this.defaultConcurrentTasks = defaultConcurrentTasks; + } + + /** + * The run duration in milliseconds. + * + * @return + */ + public Long getRunDurationMillis() { + return runDurationMillis; + } + + public void setRunDurationMillis(Long runDurationMillis) { + this.runDurationMillis = runDurationMillis; + } + + /** + * Maps default values for scheduling period for each applicable scheduling + * strategy. + * + * @return + */ + public Map getDefaultSchedulingPeriod() { + return defaultSchedulingPeriod; + } + + public void setDefaultSchedulingPeriod(Map defaultSchedulingPeriod) { + this.defaultSchedulingPeriod = defaultSchedulingPeriod; + } + + /** + * The allowable values for a property with a constrained set of options. + */ + @XmlType(name = "allowableValue") + public static class AllowableValueDTO { + + private String displayName; + private String value; + private String description; + + /** + * Returns the human-readable value that is allowed for this + * PropertyDescriptor + * + * @return + */ + public String getDisplayName() { + return displayName; + } + + public void setDisplayName(String displayName) { + this.displayName = displayName; + } + + /** + * Returns the value for this allowable value. + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + /** + * Returns a description of this Allowable Value, or null + * if no description is given + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + + if (!(obj instanceof AllowableValueDTO)) { + return false; + } + + final AllowableValueDTO other = (AllowableValueDTO) obj; + return (this.value.equals(other.getValue())); + } + + @Override + public int hashCode() { + return 23984731 + 17 * value.hashCode(); + } + } + + /** + * A description of a processor property. + */ + @XmlType(name = "propertyDescriptor") + public static class PropertyDescriptorDTO { + + private String name; + private String displayName; + private String description; + private String defaultValue; + private Set allowableValues; + private boolean required; + private boolean sensitive; + private boolean dynamic; + private boolean supportsEl; + + /** + * The set of allowable values for this property. If empty then the + * allowable values are not constrained. + * + * @return + */ + public Set getAllowableValues() { + return allowableValues; + } + + public void setAllowableValues(Set allowableValues) { + this.allowableValues = allowableValues; + } + + /** + * The default value for this property. + * + * @return + */ + public String getDefaultValue() { + return defaultValue; + } + + public void setDefaultValue(String defaultValue) { + this.defaultValue = defaultValue; + } + + /** + * And explanation of the meaning of the given property. This + * description is meant to be displayed to a user or simply provide a + * mechanism of documenting intent. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * The property name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The human-readable name to display to users. + * + * @return + */ + public String getDisplayName() { + return displayName; + } + + public void setDisplayName(String displayName) { + this.displayName = displayName; + } + + /** + * Determines whether the property is required for this processor. + * + * @return + */ + public boolean isRequired() { + return required; + } + + public void setRequired(boolean required) { + this.required = required; + } + + /** + * Indicates that the value for this property should be considered + * sensitive and protected whenever stored or represented. + * + * @return + */ + public boolean isSensitive() { + return sensitive; + } + + public void setSensitive(boolean sensitive) { + this.sensitive = sensitive; + } + + /** + * Indicates whether this property is dynamic. + * + * @return + */ + public boolean isDynamic() { + return dynamic; + } + + public void setDynamic(boolean dynamic) { + this.dynamic = dynamic; + } + + /** + * Specifies whether or not this property support expression language. + * + * @return + */ + public boolean getSupportsEl() { + return supportsEl; + } + + public void setSupportsEl(boolean supportsEl) { + this.supportsEl = supportsEl; + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java new file mode 100644 index 0000000000..71ba4edbe1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorDTO.java @@ -0,0 +1,181 @@ +/* + * 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.web.api.dto; + +import java.util.Collection; +import java.util.List; +import java.util.Map; +import javax.xml.bind.annotation.XmlType; + +/** + * Details for a processor within this NiFi. + */ +@XmlType(name = "processor") +public class ProcessorDTO extends NiFiComponentDTO { + + private String name; + private String type; + private String state; + private Map style; + private List relationships; + private String description; + private Boolean supportsParallelProcessing; + private Boolean supportsEventDriven; + + private ProcessorConfigDTO config; + + private Collection validationErrors; + + public ProcessorDTO() { + super(); + } + + /** + * The name of this processor. + * + * @return This processors name + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The type of this processor. + * + * @return This processors type + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The state of this processor. Possible states are 'RUNNING', 'STOPPED', + * and 'DISABLED'. + * + * @return + */ + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + /** + * The styles for this processor. (Currently only supports color). + * + * @return + */ + public Map getStyle() { + return style; + } + + public void setStyle(Map style) { + this.style = style; + } + + /** + * Whether this processor supports parallel processing. + * + * @return + */ + public Boolean getSupportsParallelProcessing() { + return supportsParallelProcessing; + } + + public void setSupportsParallelProcessing(Boolean supportsParallelProcessing) { + this.supportsParallelProcessing = supportsParallelProcessing; + } + + /** + * Whether this processor supports event driven scheduling. + * + * @return + */ + public Boolean getSupportsEventDriven() { + return supportsEventDriven; + } + + public void setSupportsEventDriven(Boolean supportsEventDriven) { + this.supportsEventDriven = supportsEventDriven; + } + + /** + * Gets the available relationships that this processor currently supports. + * + * @return The available relationships + */ + public List getRelationships() { + return relationships; + } + + public void setRelationships(List relationships) { + this.relationships = relationships; + } + + /** + * The configuration details for this processor. These details will be + * included in a response if the verbose flag is set to true. + * + * @return The processor configuration details + */ + public ProcessorConfigDTO getConfig() { + return config; + } + + public void setConfig(ProcessorConfigDTO config) { + this.config = config; + } + + /** + * Gets the validation errors from this processor. These validation errors + * represent the problems with the processor that must be resolved before it + * can be started. + * + * @return The validation errors + */ + public Collection getValidationErrors() { + return validationErrors; + } + + public void setValidationErrors(Collection validationErrors) { + this.validationErrors = validationErrors; + } + + /** + * Gets the description for this processor. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(final String description) { + this.description = description; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java new file mode 100644 index 0000000000..2741116a54 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/ProcessorHistoryDTO.java @@ -0,0 +1,56 @@ +/* + * 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.web.api.dto; + +import java.util.Map; +import javax.xml.bind.annotation.XmlType; + +/** + * History of a processor's properties. + */ +@XmlType(name = "processorHistory") +public class ProcessorHistoryDTO { + + private String processorId; + private Map propertyHistory; + + /** + * The processor id. + * + * @return + */ + public String getProcessorId() { + return processorId; + } + + public void setProcessorId(String processorId) { + this.processorId = processorId; + } + + /** + * The history for this processors properties. + * + * @return + */ + public Map getPropertyHistory() { + return propertyHistory; + } + + public void setPropertyHistory(Map propertyHistory) { + this.propertyHistory = propertyHistory; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java new file mode 100644 index 0000000000..064ad21b22 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/PropertyHistoryDTO.java @@ -0,0 +1,43 @@ +/* + * 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.web.api.dto; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * History of a processor property. + */ +@XmlType(name = "propertyHistory") +public class PropertyHistoryDTO { + + private List previousValues; + + /** + * The previous values. + * + * @return + */ + public List getPreviousValues() { + return previousValues; + } + + public void setPreviousValues(List previousValues) { + this.previousValues = previousValues; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java new file mode 100644 index 0000000000..7042aaa362 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RelationshipDTO.java @@ -0,0 +1,69 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details of a relationship. + */ +@XmlType(name = "relationship") +public class RelationshipDTO { + + private String name; + private String description; + private Boolean autoTerminate; + + /** + * The relationship name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The relationship description. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * Whether or not this relationship is auto terminated. + * + * @return + */ + public Boolean isAutoTerminate() { + return autoTerminate; + } + + public void setAutoTerminate(Boolean autoTerminate) { + this.autoTerminate = autoTerminate; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java new file mode 100644 index 0000000000..1e5356d209 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupContentsDTO.java @@ -0,0 +1,56 @@ +/* + * 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.web.api.dto; + +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * Contents of a remote process group. + */ +@XmlType(name = "remoteProcessGroupContents") +public class RemoteProcessGroupContentsDTO { + + private Set inputPorts; + private Set outputPorts; + + /** + * The Controller Input Ports to which data can be sent + * + * @return + */ + public Set getInputPorts() { + return inputPorts; + } + + public void setInputPorts(Set inputPorts) { + this.inputPorts = inputPorts; + } + + /** + * The Controller Output Ports from which data can be retrieved + * + * @return + */ + public Set getOutputPorts() { + return outputPorts; + } + + public void setOutputPorts(Set outputPorts) { + this.outputPorts = outputPorts; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java new file mode 100644 index 0000000000..df59b13aaa --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupDTO.java @@ -0,0 +1,279 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import java.util.List; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * Details of a remote process group in this NiFi. + */ +@XmlType(name = "remoteProcessGroup") +public class RemoteProcessGroupDTO extends NiFiComponentDTO { + + private String targetUri; + private Boolean targetSecure; + + private String name; + private String comments; + private String communicationsTimeout; + private String yieldDuration; + + private List authorizationIssues; + private Boolean transmitting; + + private Integer inputPortCount; + private Integer outputPortCount; + + private Integer activeRemoteInputPortCount; + private Integer inactiveRemoteInputPortCount; + private Integer activeRemoteOutputPortCount; + private Integer inactiveRemoteOutputPortCount; + + private Date flowRefreshed; + + private RemoteProcessGroupContentsDTO contents; + + public RemoteProcessGroupDTO() { + super(); + } + + public RemoteProcessGroupDTO(final RemoteProcessGroupDTO toCopy) { + setId(toCopy.getId()); + setPosition(toCopy.getPosition()); + targetUri = toCopy.getTargetUri(); + name = toCopy.getName(); + } + + public void setTargetUri(final String targetUri) { + this.targetUri = targetUri; + } + + /** + * The target uri of this remote process group. + * + * @return + */ + public String getTargetUri() { + return this.targetUri; + } + + /** + * The name of this remote process group. + * + * @param name + */ + public void setName(final String name) { + this.name = name; + } + + public String getName() { + return this.name; + } + + /** + * Comments for this remote process group. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * Returns any remote authorization issues for this remote process group. + * + * @return + */ + public List getAuthorizationIssues() { + return authorizationIssues; + } + + public void setAuthorizationIssues(List authorizationIssues) { + this.authorizationIssues = authorizationIssues; + } + + /** + * Whether or not this remote process group is actively transmitting. + * + * @return + */ + public Boolean isTransmitting() { + return transmitting; + } + + public void setTransmitting(Boolean transmitting) { + this.transmitting = transmitting; + } + + /** + * Whether or not the target is running securely. + * + * @return + */ + public Boolean isTargetSecure() { + return targetSecure; + } + + public void setTargetSecure(Boolean targetSecure) { + this.targetSecure = targetSecure; + } + + /** + * Returns the time period used for the timeout when communicating with this + * RemoteProcessGroup. + * + * @return + */ + public String getCommunicationsTimeout() { + return communicationsTimeout; + } + + public void setCommunicationsTimeout(String communicationsTimeout) { + this.communicationsTimeout = communicationsTimeout; + } + + /** + * When yielding, this amount of time must elaspe before this remote process + * group is scheduled again. + * + * @return + */ + public String getYieldDuration() { + return yieldDuration; + } + + public void setYieldDuration(String yieldDuration) { + this.yieldDuration = yieldDuration; + } + + /** + * The number of active remote input ports. + * + * @return + */ + public Integer getActiveRemoteInputPortCount() { + return activeRemoteInputPortCount; + } + + public void setActiveRemoteInputPortCount(Integer activeRemoteInputPortCount) { + this.activeRemoteInputPortCount = activeRemoteInputPortCount; + } + + /** + * The number of inactive remote input ports. + * + * @return + */ + public Integer getInactiveRemoteInputPortCount() { + return inactiveRemoteInputPortCount; + } + + public void setInactiveRemoteInputPortCount(Integer inactiveRemoteInputPortCount) { + this.inactiveRemoteInputPortCount = inactiveRemoteInputPortCount; + } + + /** + * The number of active remote output ports. + * + * @return + */ + public Integer getActiveRemoteOutputPortCount() { + return activeRemoteOutputPortCount; + } + + public void setActiveRemoteOutputPortCount(Integer activeRemoteOutputPortCount) { + this.activeRemoteOutputPortCount = activeRemoteOutputPortCount; + } + + /** + * The number of inactive remote output ports. + * + * @return + */ + public Integer getInactiveRemoteOutputPortCount() { + return inactiveRemoteOutputPortCount; + } + + public void setInactiveRemoteOutputPortCount(Integer inactiveRemoteOutputPortCount) { + this.inactiveRemoteOutputPortCount = inactiveRemoteOutputPortCount; + } + + /** + * The number of Remote Input Ports currently available in the remote NiFi + * instance + * + * @return + */ + public Integer getInputPortCount() { + return inputPortCount; + } + + public void setInputPortCount(Integer inputPortCount) { + this.inputPortCount = inputPortCount; + } + + /** + * The number of Remote Output Ports currently available in the remote NiFi + * instance + * + * @return + */ + public Integer getOutputPortCount() { + return outputPortCount; + } + + public void setOutputPortCount(Integer outputPortCount) { + this.outputPortCount = outputPortCount; + } + + /** + * The contents of this remote process group. Will contain available + * input/output ports. + * + * @return + */ + public RemoteProcessGroupContentsDTO getContents() { + return contents; + } + + public void setContents(RemoteProcessGroupContentsDTO contents) { + this.contents = contents; + } + + /** + * When the flow for this remote group was last refreshed. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getFlowRefreshed() { + return flowRefreshed; + } + + public void setFlowRefreshed(Date flowRefreshed) { + this.flowRefreshed = flowRefreshed; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java new file mode 100644 index 0000000000..7948dadd82 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RemoteProcessGroupPortDTO.java @@ -0,0 +1,192 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details of a port in a remote process group. + */ +@XmlType(name = "remoteProcessGroupPort") +public class RemoteProcessGroupPortDTO { + + private String id; + private String groupId; + private String name; + private String comments; + private Integer concurrentlySchedulableTaskCount; + private Boolean transmitting; + private Boolean useCompression; + private Boolean exists; + private Boolean targetRunning; + private Boolean connected; + + /** + * The comments as configured in the target port. + * + * @return + */ + public String getComments() { + return comments; + } + + public void setComments(String comments) { + this.comments = comments; + } + + /** + * The number tasks that may transmit flow files to the target port + * concurrently. + * + * @return + */ + public Integer getConcurrentlySchedulableTaskCount() { + return concurrentlySchedulableTaskCount; + } + + public void setConcurrentlySchedulableTaskCount(Integer concurrentlySchedulableTaskCount) { + this.concurrentlySchedulableTaskCount = concurrentlySchedulableTaskCount; + } + + /** + * The id of the target port. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The id of the remote process group that this port resides in. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The name of the target port. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * Whether or not this remote group port is configured for transmission. + * + * @return + */ + public Boolean isTransmitting() { + return transmitting; + } + + public void setTransmitting(Boolean transmitting) { + this.transmitting = transmitting; + } + + /** + * Whether or not flow file are compressed when sent to this target port. + * + * @return + */ + public Boolean getUseCompression() { + return useCompression; + } + + public void setUseCompression(Boolean useCompression) { + this.useCompression = useCompression; + } + + /** + * Whether or not the target port exists. + * + * @return + */ + public Boolean getExists() { + return exists; + } + + public void setExists(Boolean exists) { + this.exists = exists; + } + + /** + * Whether or not the target port is running. + * + * @return + */ + public Boolean isTargetRunning() { + return targetRunning; + } + + public void setTargetRunning(Boolean targetRunning) { + this.targetRunning = targetRunning; + } + + /** + * Whether or not this port has either an incoming or outgoing connection. + * + * @return + */ + public Boolean isConnected() { + return connected; + } + + public void setConnected(Boolean connected) { + this.connected = connected; + } + + @Override + public int hashCode() { + return 923847 + String.valueOf(name).hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (obj == null) { + return false; + } + if (!(obj instanceof RemoteProcessGroupPortDTO)) { + return false; + } + final RemoteProcessGroupPortDTO other = (RemoteProcessGroupPortDTO) obj; + if (name == null && other.name == null) { + return true; + } + + if (name == null) { + return false; + } + return name.equals(other.name); + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java new file mode 100644 index 0000000000..e608a7efcb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/RevisionDTO.java @@ -0,0 +1,63 @@ +/* + * 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.web.api.dto; + +import javax.xml.bind.annotation.XmlType; + +/** + * Current revision for this NiFi. + */ +@XmlType(name = "revision") +public class RevisionDTO { + + private String clientId; + private Long version; + + /* getters / setters */ + /** + * A client identifier used to make a request. By including a client + * identifier, the API can allow multiple requests without needing the + * current revision. Due to the asynchronous nature of requests/responses + * this was implemented to allow the client to make numerous requests + * without having to wait for the previous response to come back. + * + * @return The client id + */ + public String getClientId() { + return clientId; + } + + public void setClientId(String clientId) { + this.clientId = clientId; + } + + /** + * NiFi employs an optimistic locking strategy where the client must include + * a revision in their request when performing an update. In a response, + * this field represents the updated base version. + * + * @return The revision + */ + public Long getVersion() { + return version; + } + + public void setVersion(Long version) { + this.version = version; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java new file mode 100644 index 0000000000..2ee13106da --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SnippetDTO.java @@ -0,0 +1,239 @@ +/* + * 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.web.api.dto; + +import java.util.HashSet; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * The contents of a snippet of a flow. + */ +@XmlType(name = "snippet") +public class SnippetDTO { + + private String id; + private String uri; + private String parentGroupId; + private Boolean linked; + + // when specified these are only considered during creation + private Set processGroups = new HashSet<>(); + private Set remoteProcessGroups = new HashSet<>(); + private Set processors = new HashSet<>(); + private Set inputPorts = new HashSet<>(); + private Set outputPorts = new HashSet<>(); + private Set connections = new HashSet<>(); + private Set labels = new HashSet<>(); + private Set funnels = new HashSet<>(); + + private FlowSnippetDTO contents; + + /** + * The id of this snippet. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The uri of this snippet. + * + * @return + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + /** + * The group id for the components in this snippet. + * + * @return + */ + public String getParentGroupId() { + return parentGroupId; + } + + public void setParentGroupId(String parentGroupId) { + this.parentGroupId = parentGroupId; + } + + /** + * Whether or not this snippet is linked to the underlying data flow. + * + * @return + */ + public Boolean isLinked() { + return linked; + } + + public void setLinked(Boolean linked) { + this.linked = linked; + } + + /** + * The ids of the connections in this snippet. These ids will be populated + * within each response. They can be specified when creating a snippet. + * However, once a snippet has been created its contents cannot be modified + * (these ids are ignored during update requests). + * + * @return + */ + public Set getConnections() { + return connections; + } + + public void setConnections(Set connections) { + this.connections = connections; + } + + /** + * The ids of the funnels in this snippet. These ids will be populated + * within each response. They can be specified when creating a snippet. + * However, once a snippet has been created its contents cannot be modified + * (these ids are ignored during update requests). + * + * @param funnels + */ + public Set getFunnels() { + return funnels; + } + + public void setFunnels(Set funnels) { + this.funnels = funnels; + } + + /** + * The ids of the input port in this snippet. These ids will be populated + * within each response. They can be specified when creating a snippet. + * However, once a snippet has been created its contents cannot be modified + * (these ids are ignored during update requests). + * + * @return + */ + public Set getInputPorts() { + return inputPorts; + } + + public void setInputPorts(Set inputPorts) { + this.inputPorts = inputPorts; + } + + /** + * The ids of the labels in this snippet. These ids will be populated within + * each response. They can be specified when creating a snippet. However, + * once a snippet has been created its contents cannot be modified (these + * ids are ignored during update requests). + * + * @return + */ + public Set getLabels() { + return labels; + } + + public void setLabels(Set labels) { + this.labels = labels; + } + + /** + * The ids of the output ports in this snippet. These ids will be populated + * within each response. They can be specified when creating a snippet. + * However, once a snippet has been created its contents cannot be modified + * (these ids are ignored during update requests). + * + * @return + */ + public Set getOutputPorts() { + return outputPorts; + } + + public void setOutputPorts(Set outputPorts) { + this.outputPorts = outputPorts; + } + + /** + * The ids of the process groups in this snippet. These ids will be + * populated within each response. They can be specified when creating a + * snippet. However, once a snippet has been created its contents cannot be + * modified (these ids are ignored during update requests). + * + * @return + */ + public Set getProcessGroups() { + return processGroups; + } + + public void setProcessGroups(Set processGroups) { + this.processGroups = processGroups; + } + + /** + * The ids of the processors in this snippet. These ids will be populated + * within each response. They can be specified when creating a snippet. + * However, once a snippet has been created its contents cannot be modified + * (these ids are ignored during update requests). + * + * @return + */ + public Set getProcessors() { + return processors; + } + + public void setProcessors(Set processors) { + this.processors = processors; + } + + /** + * The ids of the remote process groups in this snippet. These ids will be + * populated within each response. They can be specified when creating a + * snippet. However, once a snippet has been created its contents cannot be + * modified (these ids are ignored during update requests). + * + * @return + */ + public Set getRemoteProcessGroups() { + return remoteProcessGroups; + } + + public void setRemoteProcessGroups(Set remoteProcessGroups) { + this.remoteProcessGroups = remoteProcessGroups; + } + + /** + * The contents of the configuration for this snippet. + * + * @return + */ + public FlowSnippetDTO getContents() { + return contents; + } + + public void setContents(FlowSnippetDTO contents) { + this.contents = contents; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java new file mode 100644 index 0000000000..0b2085298f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/SystemDiagnosticsDTO.java @@ -0,0 +1,461 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * The diagnostics of the system this NiFi is running on. + */ +@XmlType(name = "systemDiagnostics") +public class SystemDiagnosticsDTO { + + private String totalNonHeap; + private String usedNonHeap; + private String freeNonHeap; + private String maxNonHeap; + private String nonHeapUtilization; + + private String totalHeap; + private String usedHeap; + private String freeHeap; + private String maxHeap; + private String heapUtilization; + + private Integer availableProcessors; + private Double processorLoadAverage; + + private Integer totalThreads; + private Integer daemonThreads; + + private StorageUsageDTO flowFileRepositoryStorageUsage; + private Set contentRepositoryStorageUsage; + private Set garbageCollection; + + private Date statsLastRefreshed; + + /* getters / setters */ + /** + * The number of available processors, if supported. + * + * @return + */ + public Integer getAvailableProcessors() { + return availableProcessors; + } + + public void setAvailableProcessors(Integer availableProcessors) { + this.availableProcessors = availableProcessors; + } + + /** + * The number of daemon threads. + * + * @return + */ + public Integer getDaemonThreads() { + return daemonThreads; + } + + public void setDaemonThreads(Integer daemonThreads) { + this.daemonThreads = daemonThreads; + } + + /** + * The amount of free heap. + * + * @return + */ + public String getFreeHeap() { + return freeHeap; + } + + public void setFreeHeap(String freeHeap) { + this.freeHeap = freeHeap; + } + + /** + * The amount of free non-heap. + * + * @return + */ + public String getFreeNonHeap() { + return freeNonHeap; + } + + public void setFreeNonHeap(String freeNonHeap) { + this.freeNonHeap = freeNonHeap; + } + + /** + * The max size of the heap. + * + * @return + */ + public String getMaxHeap() { + return maxHeap; + } + + public void setMaxHeap(String maxHeap) { + this.maxHeap = maxHeap; + } + + /** + * The max size of the non-heap. + * + * @return + */ + public String getMaxNonHeap() { + return maxNonHeap; + } + + public void setMaxNonHeap(String maxNonHeap) { + this.maxNonHeap = maxNonHeap; + } + + /** + * The processor load average, if supported. + * + * @return + */ + public Double getProcessorLoadAverage() { + return processorLoadAverage; + } + + public void setProcessorLoadAverage(Double processorLoadAverage) { + this.processorLoadAverage = processorLoadAverage; + } + + /** + * The total size of the heap. + * + * @return + */ + public String getTotalHeap() { + return totalHeap; + } + + public void setTotalHeap(String totalHeap) { + this.totalHeap = totalHeap; + } + + /** + * The total size of non-heap. + * + * @return + */ + public String getTotalNonHeap() { + return totalNonHeap; + } + + public void setTotalNonHeap(String totalNonHeap) { + this.totalNonHeap = totalNonHeap; + } + + /** + * The total number of threads. + * + * @return + */ + public Integer getTotalThreads() { + return totalThreads; + } + + public void setTotalThreads(Integer totalThreads) { + this.totalThreads = totalThreads; + } + + /** + * The amount of used heap. + * + * @return + */ + public String getUsedHeap() { + return usedHeap; + } + + public void setUsedHeap(String usedHeap) { + this.usedHeap = usedHeap; + } + + /** + * The amount of used non-heap. + * + * @return + */ + public String getUsedNonHeap() { + return usedNonHeap; + } + + public void setUsedNonHeap(String usedNonHeap) { + this.usedNonHeap = usedNonHeap; + } + + /** + * The heap utilization. + * + * @return + */ + public String getHeapUtilization() { + return heapUtilization; + } + + public void setHeapUtilization(String heapUtilization) { + this.heapUtilization = heapUtilization; + } + + /** + * The non-heap utilization. + * + * @return + */ + public String getNonHeapUtilization() { + return nonHeapUtilization; + } + + public void setNonHeapUtilization(String nonHeapUsage) { + this.nonHeapUtilization = nonHeapUsage; + } + + /** + * The content repository storage usage. + * + * @return + */ + public Set getContentRepositoryStorageUsage() { + return contentRepositoryStorageUsage; + } + + public void setContentRepositoryStorageUsage(Set contentRepositoryStorageUsage) { + this.contentRepositoryStorageUsage = contentRepositoryStorageUsage; + } + + /** + * The flowfile repository storage usage. + * + * @return + */ + public StorageUsageDTO getFlowFileRepositoryStorageUsage() { + return flowFileRepositoryStorageUsage; + } + + public void setFlowFileRepositoryStorageUsage(StorageUsageDTO flowFileRepositoryStorageUsage) { + this.flowFileRepositoryStorageUsage = flowFileRepositoryStorageUsage; + } + + /** + * Garbage collection details. + * + * @return + */ + public Set getGarbageCollection() { + return garbageCollection; + } + + public void setGarbageCollection(Set garbageCollection) { + this.garbageCollection = garbageCollection; + } + + /** + * When these diagnostics were generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + + /** + * Details of storage usage. + */ + @XmlType(name = "storageUsage") + public static class StorageUsageDTO { + + private String identifier; + private String freeSpace; + private String totalSpace; + private String usedSpace; + private Long freeSpaceBytes; + private Long totalSpaceBytes; + private Long usedSpaceBytes; + private String utilization; + + /** + * The identifier for this storage location. + * + * @return + */ + public String getIdentifier() { + return identifier; + } + + public void setIdentifier(String identifier) { + this.identifier = identifier; + } + + /** + * The amount of free space. + * + * @return + */ + public String getFreeSpace() { + return freeSpace; + } + + public void setFreeSpace(String freeSpace) { + this.freeSpace = freeSpace; + } + + /** + * The amount of total space. + * + * @param freeSpace + */ + public String getTotalSpace() { + return totalSpace; + } + + public void setTotalSpace(String totalSpace) { + this.totalSpace = totalSpace; + } + + /** + * The amount of used space. + * + * @return + */ + public String getUsedSpace() { + return usedSpace; + } + + public void setUsedSpace(String usedSpace) { + this.usedSpace = usedSpace; + } + + /** + * The utilization of this storage location. + * + * @return + */ + public String getUtilization() { + return utilization; + } + + public void setUtilization(String utilization) { + this.utilization = utilization; + } + + /** + * The number of bytes of free space. + * + * @return + */ + public Long getFreeSpaceBytes() { + return freeSpaceBytes; + } + + public void setFreeSpaceBytes(Long freeSpaceBytes) { + this.freeSpaceBytes = freeSpaceBytes; + } + + /** + * The number of bytes of total space. + * + * @return + */ + public Long getTotalSpaceBytes() { + return totalSpaceBytes; + } + + public void setTotalSpaceBytes(Long totalSpaceBytes) { + this.totalSpaceBytes = totalSpaceBytes; + } + + /** + * The number of bytes of used space. + * + * @return + */ + public Long getUsedSpaceBytes() { + return usedSpaceBytes; + } + + public void setUsedSpaceBytes(Long usedSpaceBytes) { + this.usedSpaceBytes = usedSpaceBytes; + } + } + + /** + * Details for garbage collection. + */ + @XmlType(name = "garbageCollection") + public static class GarbageCollectionDTO { + + private String name; + private long collectionCount; + private String collectionTime; + + /** + * The name of the garbage collector. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + public long getCollectionCount() { + return collectionCount; + } + + /** + * The number of times garbage collection has run. + * + * @param collectionCount + */ + public void setCollectionCount(long collectionCount) { + this.collectionCount = collectionCount; + } + + /** + * The total amount of time spent garbage collecting. + * + * @return + */ + public String getCollectionTime() { + return collectionTime; + } + + public void setCollectionTime(String collectionTime) { + this.collectionTime = collectionTime; + } + + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java new file mode 100644 index 0000000000..156a6e9382 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/TemplateDTO.java @@ -0,0 +1,117 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * Defines a template. + */ +@XmlRootElement(name = "template") +public class TemplateDTO { + + private String uri; + + private String id; + private String name; + private String description; + private Date timestamp; + + private FlowSnippetDTO snippet; + + /** + * The id for this template. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The uri for this template. + * + * @return + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + /** + * The name of this template. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The description of this template. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * The timestamp when this template was created. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + /** + * The snippet in this template. + * + * @return + */ + public FlowSnippetDTO getSnippet() { + return snippet; + } + + public void setSnippet(FlowSnippetDTO snippet) { + this.snippet = snippet; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java new file mode 100644 index 0000000000..85c82dc3a0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserDTO.java @@ -0,0 +1,177 @@ +/* + * 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.web.api.dto; + +import java.util.Date; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * A user of this NiFi. + */ +@XmlType(name = "user") +public class UserDTO { + + private String id; + private String dn; + private String userName; + private String userGroup; + private String justification; + private Date creation; + private String status; + + private Date lastVerified; + private Date lastAccessed; + private Set authorities; + + /** + * The user id. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The users authorities. + * + * @return + */ + public Set getAuthorities() { + return authorities; + } + + public void setAuthorities(Set authorities) { + this.authorities = authorities; + } + + /** + * The creation time for this users account. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getCreation() { + return creation; + } + + public void setCreation(Date creation) { + this.creation = creation; + } + + /** + * The users DN. + * + * @return + */ + public String getDn() { + return dn; + } + + public void setDn(String dn) { + this.dn = dn; + } + + /** + * The users name. If the name could not be extracted from the DN, this + * value will be the entire DN. + * + * @return + */ + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + /** + * The user group. + * + * @return + */ + public String getUserGroup() { + return userGroup; + } + + public void setUserGroup(String userGroup) { + this.userGroup = userGroup; + } + + /** + * The users account justification. + * + * @return + */ + public String getJustification() { + return justification; + } + + public void setJustification(String justification) { + this.justification = justification; + } + + /** + * The time that the user last accessed the system. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getLastAccessed() { + return lastAccessed; + } + + public void setLastAccessed(Date lastAccessed) { + this.lastAccessed = lastAccessed; + } + + /** + * The time that the users credentials were last verified. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getLastVerified() { + return lastVerified; + } + + public void setLastVerified(Date lastVerified) { + this.lastVerified = lastVerified; + } + + /** + * The status of the users account. + * + * @return + */ + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java new file mode 100644 index 0000000000..285c355841 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/UserGroupDTO.java @@ -0,0 +1,84 @@ +/* + * 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.web.api.dto; + +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * A user group in this NiFi. + */ +@XmlType(name = "userGroup") +public class UserGroupDTO { + + private String group; + private Set userIds; + private Set authorities; + private String status; + + /** + * The user group. + * + * @return + */ + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + + /** + * The users in this group. + * + * @return + */ + public Set getUserIds() { + return userIds; + } + + public void setUserIds(Set userIds) { + this.userIds = userIds; + } + + /** + * The status of the users account. + * + * @return + */ + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } + + /** + * The users authorities. + * + * @return + */ + public Set getAuthorities() { + return authorities; + } + + public void setAuthorities(Set authorities) { + this.authorities = authorities; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java new file mode 100644 index 0000000000..84024801f2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/ActionDTO.java @@ -0,0 +1,176 @@ +/* + * 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.web.api.dto.action; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.action.component.details.ComponentDetailsDTO; +import org.apache.nifi.web.api.dto.action.details.ActionDetailsDTO; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * An action performed in this NiFi. + */ +@XmlType(name = "action") +public class ActionDTO { + + private Integer id; + private String userDn; + private String userName; + private Date timestamp; + + private String sourceId; + private String sourceName; + private String sourceType; + private ComponentDetailsDTO componentDetails; + + private String operation; + private ActionDetailsDTO actionDetails; + + /** + * The action id. + * + * @return + */ + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + + /** + * The user dn who perform this action. + * + * @return + */ + public String getUserDn() { + return userDn; + } + + public void setUserDn(String userDn) { + this.userDn = userDn; + } + + /** + * The user name who perform this action. + * + * @return + */ + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + /** + * This action's timestamp. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + /** + * The id of the source component of this action. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The name of the source component of this action. + * + * @return + */ + public String getSourceName() { + return sourceName; + } + + public void setSourceName(String sourceName) { + this.sourceName = sourceName; + } + + /** + * The type of the source component of this action. + * + * @return + */ + public String getSourceType() { + return sourceType; + } + + public void setSourceType(String sourceType) { + this.sourceType = sourceType; + } + + /** + * The component details (if any) for this action. + * + * @return + */ + public ComponentDetailsDTO getComponentDetails() { + return componentDetails; + } + + public void setComponentDetails(ComponentDetailsDTO componentDetails) { + this.componentDetails = componentDetails; + } + + /** + * The operation being performed in this action. + * + * @return + */ + public String getOperation() { + return operation; + } + + public void setOperation(String operation) { + this.operation = operation; + } + + /** + * The action details (if any) for this action. + * + * @return + */ + public ActionDetailsDTO getActionDetails() { + return actionDetails; + } + + public void setActionDetails(ActionDetailsDTO actionDetails) { + this.actionDetails = actionDetails; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java new file mode 100644 index 0000000000..9ab47ed566 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryDTO.java @@ -0,0 +1,74 @@ +/* + * 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.web.api.dto.action; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * NiFi action history. + */ +@XmlType(name = "history") +public class HistoryDTO { + + private Integer total; + private Date lastRefreshed; + private Collection actions; + + /** + * The total number of actions. + * + * @return + */ + public Integer getTotal() { + return total; + } + + public void setTotal(Integer total) { + this.total = total; + } + + /** + * Timestamp when these records were returned. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getLastRefreshed() { + return lastRefreshed; + } + + public void setLastRefreshed(Date lastRefreshed) { + this.lastRefreshed = lastRefreshed; + } + + /** + * The actions for this range. + * + * @return + */ + public Collection getActions() { + return actions; + } + + public void setActions(Collection actions) { + this.actions = actions; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java new file mode 100644 index 0000000000..e8e11e5a05 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/HistoryQueryDTO.java @@ -0,0 +1,144 @@ +/* + * 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.web.api.dto.action; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * A history query to find desired actions. + */ +@XmlType(name = "historyQuery") +public class HistoryQueryDTO { + + private String userName; + private String sourceId; + private Date startDate; + private Date endDate; + private Integer offset; + private Integer count; + private String sortColumn; + private String sortOrder; + + /** + * The user name. + * + * @return + */ + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + + /** + * The source component id. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The start date. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getStartDate() { + return startDate; + } + + public void setStartDate(Date startDate) { + this.startDate = startDate; + } + + /** + * The end date. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getEndDate() { + return endDate; + } + + public void setEndDate(Date endDate) { + this.endDate = endDate; + } + + /** + * The offset. + * + * @return + */ + public Integer getOffset() { + return offset; + } + + public void setOffset(Integer offset) { + this.offset = offset; + } + + /** + * The desired row count. + * + * @return + */ + public Integer getCount() { + return count; + } + + public void setCount(Integer count) { + this.count = count; + } + + /** + * The desired sort column. + * + * @return + */ + public String getSortColumn() { + return sortColumn; + } + + public void setSortColumn(String sortColumn) { + this.sortColumn = sortColumn; + } + + /** + * The desired sort order. + * + * @return + */ + public String getSortOrder() { + return sortOrder; + } + + public void setSortOrder(String sortOrder) { + this.sortOrder = sortOrder; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java new file mode 100644 index 0000000000..58086ce285 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ComponentDetailsDTO.java @@ -0,0 +1,32 @@ +/* + * 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.web.api.dto.action.component.details; + +import javax.xml.bind.annotation.XmlSeeAlso; +import javax.xml.bind.annotation.XmlType; + +/** + * + */ +@XmlType(name = "componentDetails") +@XmlSeeAlso({ + ProcessorDetailsDTO.class, + RemoteProcessGroupDetailsDTO.class +}) +public class ComponentDetailsDTO { + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java new file mode 100644 index 0000000000..3523f62e94 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/ProcessorDetailsDTO.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto.action.component.details; + +import javax.xml.bind.annotation.XmlType; + +/** + * Processor details for an action. + */ +@XmlType(name = "processorDetails") +public class ProcessorDetailsDTO extends ComponentDetailsDTO { + + private String type; + + /** + * The processors type. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java new file mode 100644 index 0000000000..280939808d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/component/details/RemoteProcessGroupDetailsDTO.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto.action.component.details; + +import javax.xml.bind.annotation.XmlType; + +/** + * Remote process group details for an action. + */ +@XmlType(name = "remoteProcessGroupDetails") +public class RemoteProcessGroupDetailsDTO extends ComponentDetailsDTO { + + private String uri; + + /** + * The URI of the remote process group. + * + * @return + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java new file mode 100644 index 0000000000..4074cbafcd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ActionDetailsDTO.java @@ -0,0 +1,34 @@ +/* + * 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.web.api.dto.action.details; + +import javax.xml.bind.annotation.XmlSeeAlso; +import javax.xml.bind.annotation.XmlType; + +/** + * + */ +@XmlType(name = "details") +@XmlSeeAlso({ + ConfigureDetailsDTO.class, + MoveDetailsDTO.class, + ConnectDetailsDTO.class, + PurgeDetailsDTO.class +}) +public class ActionDetailsDTO { + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java new file mode 100644 index 0000000000..4011b007b6 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConfigureDetailsDTO.java @@ -0,0 +1,69 @@ +/* + * 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.web.api.dto.action.details; + +import javax.xml.bind.annotation.XmlType; + +/** + * Configuration details for an Action. + */ +@XmlType(name = "configureDetails") +public class ConfigureDetailsDTO extends ActionDetailsDTO { + + private String name; + private String previousValue; + private String value; + + /** + * The name of the property that was modified. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The previous value. + * + * @return + */ + public String getPreviousValue() { + return previousValue; + } + + public void setPreviousValue(String previousValue) { + this.previousValue = previousValue; + } + + /** + * The new value. + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java new file mode 100644 index 0000000000..ba88bd10da --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/ConnectDetailsDTO.java @@ -0,0 +1,126 @@ +/* + * 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.web.api.dto.action.details; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details for connect Actions. + */ +@XmlType(name = "connectDetails") +public class ConnectDetailsDTO extends ActionDetailsDTO { + + private String sourceId; + private String sourceName; + private String sourceType; + private String relationship; + private String destinationId; + private String destinationName; + private String destinationType; + + /** + * The id of the source of the connection. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The name of the source of the connection. + * + * @return + */ + public String getSourceName() { + return sourceName; + } + + public void setSourceName(String sourceName) { + this.sourceName = sourceName; + } + + /** + * The type of the source of the connection. + * + * @return + */ + public String getSourceType() { + return sourceType; + } + + public void setSourceType(String sourceType) { + this.sourceType = sourceType; + } + + /** + * The name of the relationship that was connected. + * + * @return + */ + public String getRelationship() { + return relationship; + } + + public void setRelationship(String relationship) { + this.relationship = relationship; + } + + /** + * The id of the destination of the connection. + * + * @return + */ + public String getDestinationId() { + return destinationId; + } + + public void setDestinationId(String destinationId) { + this.destinationId = destinationId; + } + + /** + * The name of the destination of the connection. + * + * @return + */ + public String getDestinationName() { + return destinationName; + } + + public void setDestinationName(String destinationName) { + this.destinationName = destinationName; + } + + /** + * The type of the destination of the connection. + * + * @return + */ + public String getDestinationType() { + return destinationType; + } + + public void setDestinationType(String destinationType) { + this.destinationType = destinationType; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java new file mode 100644 index 0000000000..c6f04509d6 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/MoveDetailsDTO.java @@ -0,0 +1,83 @@ +/* + * 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.web.api.dto.action.details; + +import javax.xml.bind.annotation.XmlType; + +/** + * Details of the move operation. + */ +@XmlType(name = "moveDetails") +public class MoveDetailsDTO extends ActionDetailsDTO { + + private String previousGroupId; + private String previousGroup; + private String groupId; + private String group; + + /** + * The id of the group the components previously belonged to. + * + * @return + */ + public String getPreviousGroupId() { + return previousGroupId; + } + + public void setPreviousGroupId(String previousGroupId) { + this.previousGroupId = previousGroupId; + } + + /** + * The name of the group of the components previously belonged to. + * + * @return + */ + public String getPreviousGroup() { + return previousGroup; + } + + public void setPreviousGroup(String previousGroup) { + this.previousGroup = previousGroup; + } + + /** + * The id of the group the components belong to. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The name of the group the components belong to. + * + * @return + */ + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java new file mode 100644 index 0000000000..b5a5bcbadf --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/action/details/PurgeDetailsDTO.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.dto.action.details; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * Details of a purge operation. + */ +@XmlType(name = "purgeDetails") +public class PurgeDetailsDTO extends ActionDetailsDTO { + + private Date endDate; + + /** + * The end date for this purge action. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getEndDate() { + return endDate; + } + + public void setEndDate(Date endDate) { + this.endDate = endDate; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java new file mode 100644 index 0000000000..ed5807799c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/AttributeDTO.java @@ -0,0 +1,69 @@ +/* + * 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.web.api.dto.provenance; + +import javax.xml.bind.annotation.XmlType; + +/** + * Represents a processor's attribute for a provenance event. + */ +@XmlType(name = "attribute") +public class AttributeDTO { + + private String name; + private String value; + private String previousValue; + + /** + * The attribute name. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The attribute value. + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + /** + * The value of this attribute before the event took place. + * + * @return + */ + public String getPreviousValue() { + return previousValue; + } + + public void setPreviousValue(String previousValue) { + this.previousValue = previousValue; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java new file mode 100644 index 0000000000..90dd119dca --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceDTO.java @@ -0,0 +1,165 @@ +/* + * 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.web.api.dto.provenance; + +import java.util.Date; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.web.api.dto.util.TimestampAdapter; + +/** + * A provenance submission. Incorporates the request, its current status, and + * the results. + */ +@XmlType(name = "provenance") +public class ProvenanceDTO { + + private String id; + private String uri; + private String clusterNodeId; + + private Date submissionTime; + private Date expiration; + + private Integer percentCompleted; + private Boolean finished; + + private ProvenanceRequestDTO request; + private ProvenanceResultsDTO results; + + /** + * The id of this provenance query. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The URI for this query. Used for obtaining the requests at a later time. + * + * @return + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + /** + * The id of the node in the cluster where this provenance originated. + * + * @return + */ + public String getClusterNodeId() { + return clusterNodeId; + } + + public void setClusterNodeId(String clusterNodeId) { + this.clusterNodeId = clusterNodeId; + } + + /** + * The time the query was submitted. + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getSubmissionTime() { + return submissionTime; + } + + public void setSubmissionTime(Date submissionTime) { + this.submissionTime = submissionTime; + } + + /** + * The expiration time of the query results. + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getExpiration() { + return expiration; + } + + public void setExpiration(Date expiration) { + this.expiration = expiration; + } + + /** + * The percent completed. + * + * @return + */ + public Integer getPercentCompleted() { + return percentCompleted; + } + + public void setPercentCompleted(Integer percentCompleted) { + this.percentCompleted = percentCompleted; + } + + /** + * Whether the query has finished. + * + * @return + */ + public Boolean isFinished() { + return finished; + } + + public void setFinished(Boolean finished) { + this.finished = finished; + } + + /** + * The provenance request. + * + * @return + */ + public ProvenanceRequestDTO getRequest() { + return request; + } + + public void setRequest(ProvenanceRequestDTO request) { + this.request = request; + } + + /** + * The results of this query. + * + * @return + */ + public ProvenanceResultsDTO getResults() { + return results; + } + + public void setResults(ProvenanceResultsDTO results) { + this.results = results; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java new file mode 100644 index 0000000000..b1ead42c65 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceEventDTO.java @@ -0,0 +1,630 @@ +/* + * 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.web.api.dto.provenance; + +import java.util.Collection; +import java.util.Date; +import java.util.List; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.web.api.dto.util.TimestampAdapter; + +/** + * A provenance event. + */ +@XmlType(name = "provenanceEvent") +public class ProvenanceEventDTO { + + private String id; + + // in search results table + private Long eventId; + private Date eventTime; + private Long eventDuration; + private Long lineageDuration; + private String eventType; + private String flowFileUuid; + private String fileSize; + private Long fileSizeBytes; + private String clusterNodeId; // include when clustered + private String clusterNodeAddress; // include when clustered + + private String groupId; + private String componentId; + private String componentType; + private String componentName; + private String sourceSystemFlowFileId; + private String alternateIdentifierUri; + private Collection attributes; + private List parentUuids; + private List childUuids; + + private String transitUri; + + private String relationship; + private String details; + + // content + private Boolean contentEqual; + private Boolean inputContentAvailable; + private String inputContentClaimSection; + private String inputContentClaimContainer; + private String inputContentClaimIdentifier; + private Long inputContentClaimOffset; + private String inputContentClaimFileSize; + private Long inputContentClaimFileSizeBytes; + private Boolean outputContentAvailable; + private String outputContentClaimSection; + private String outputContentClaimContainer; + private String outputContentClaimIdentifier; + private Long outputContentClaimOffset; + private String outputContentClaimFileSize; + private Long outputContentClaimFileSizeBytes; + + // replay + private Boolean replayAvailable; + private String replayExplanation; + private String sourceConnectionIdentifier; + + /** + * The event uuid. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The event id. + * + * @return + */ + public Long getEventId() { + return eventId; + } + + public void setEventId(Long eventId) { + this.eventId = eventId; + } + + /** + * The time the event occurred. + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getEventTime() { + return eventTime; + } + + public void setEventTime(Date eventTime) { + this.eventTime = eventTime; + } + + /** + * The UUID of the FlowFile for this event. + * + * @return + */ + public String getFlowFileUuid() { + return flowFileUuid; + } + + public void setFlowFileUuid(String flowFileUuid) { + this.flowFileUuid = flowFileUuid; + } + + /** + * The size of the FlowFile for this event. + * + * @return + */ + public String getFileSize() { + return fileSize; + } + + public void setFileSize(String fileSize) { + this.fileSize = fileSize; + } + + /** + * The size of the FlowFile in bytes for this event. + * + * @return + */ + public Long getFileSizeBytes() { + return fileSizeBytes; + } + + public void setFileSizeBytes(Long fileSizeBytes) { + this.fileSizeBytes = fileSizeBytes; + } + + /** + * The type of this event. + * + * @return + */ + public String getEventType() { + return eventType; + } + + public void setEventType(String eventType) { + this.eventType = eventType; + } + + /** + * The attributes for the FlowFile for this event. + * + * @return + */ + public Collection getAttributes() { + return attributes; + } + + public void setAttributes(Collection attributes) { + this.attributes = attributes; + } + + /** + * The id of the group that this component resides in. If the component is + * no longer in the flow, the group id will not be set. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The id of the component that generated this event. + * + * @return + */ + public String getComponentId() { + return componentId; + } + + public void setComponentId(String componentId) { + this.componentId = componentId; + } + + /** + * The name of the component that generated this event. + * + * @return + */ + public String getComponentName() { + return componentName; + } + + public void setComponentName(String componentName) { + this.componentName = componentName; + } + + /** + * The type of the component that generated this event. + * + * @return + */ + public String getComponentType() { + return componentType; + } + + public void setComponentType(String componentType) { + this.componentType = componentType; + } + + /** + * The source/destination system URI if the event was a RECEIVE/SEND. + * + * @return + */ + public String getTransitUri() { + return transitUri; + } + + public void setTransitUri(String transitUri) { + this.transitUri = transitUri; + } + + /** + * The alternate identifier URI for the FlowFile for this event. + * + * @return + */ + public String getAlternateIdentifierUri() { + return alternateIdentifierUri; + } + + public void setAlternateIdentifierUri(String alternateIdentifierUri) { + this.alternateIdentifierUri = alternateIdentifierUri; + } + + /** + * The identifier of the node where this event originated. + * + * @return + */ + public String getClusterNodeId() { + return clusterNodeId; + } + + public void setClusterNodeId(String clusterNodeId) { + this.clusterNodeId = clusterNodeId; + } + + /** + * The label to use to show which node this event originated from. + * + * @return + */ + public String getClusterNodeAddress() { + return clusterNodeAddress; + } + + public void setClusterNodeAddress(String clusterNodeAddress) { + this.clusterNodeAddress = clusterNodeAddress; + } + + /** + * The parent uuids for this event. + * + * @return + */ + public List getParentUuids() { + return parentUuids; + } + + public void setParentUuids(List parentUuids) { + this.parentUuids = parentUuids; + } + + /** + * The child uuids for this event. + * + * @return + */ + public List getChildUuids() { + return childUuids; + } + + public void setChildUuids(List childUuids) { + this.childUuids = childUuids; + } + + /** + * The duration of the event, in milliseconds. + * + * @return + */ + public Long getEventDuration() { + return eventDuration; + } + + public void setEventDuration(Long eventDuration) { + this.eventDuration = eventDuration; + } + + /** + * The duration since the lineage began, in milliseconds. + * + * @return + */ + public Long getLineageDuration() { + return lineageDuration; + } + + public void setLineageDuration(Long lineageDuration) { + this.lineageDuration = lineageDuration; + } + + /** + * The source system FlowFile id. + * + * @return + */ + public String getSourceSystemFlowFileId() { + return sourceSystemFlowFileId; + } + + public void setSourceSystemFlowFileId(String sourceSystemFlowFileId) { + this.sourceSystemFlowFileId = sourceSystemFlowFileId; + } + + /** + * If this represents a route event, this is the relationship to which the + * flowfile was routed. + * + * @return + */ + public String getRelationship() { + return relationship; + } + + public void setRelationship(String relationship) { + this.relationship = relationship; + } + + /** + * The event details. + * + * @return + */ + public String getDetails() { + return details; + } + + public void setDetails(String details) { + this.details = details; + } + + /** + * Whether or not the input and output content claim is the same. + * + * @return + */ + public Boolean getContentEqual() { + return contentEqual; + } + + public void setContentEqual(Boolean contentEqual) { + this.contentEqual = contentEqual; + } + + /** + * Returns whether or not the output content is still available. + * + * @return + */ + public Boolean getOutputContentAvailable() { + return outputContentAvailable; + } + + public void setOutputContentAvailable(Boolean outputContentAvailable) { + this.outputContentAvailable = outputContentAvailable; + } + + /** + * Returns the Section in which the output Content Claim lives, or + * null if no Content Claim exists. + * + * @return + */ + public String getOutputContentClaimSection() { + return outputContentClaimSection; + } + + public void setOutputContentClaimSection(String contentClaimSection) { + this.outputContentClaimSection = contentClaimSection; + } + + /** + * Returns the Container in which the output Content Claim lives, or + * null if no Content Claim exists. + * + * @return + */ + public String getOutputContentClaimContainer() { + return outputContentClaimContainer; + } + + public void setOutputContentClaimContainer(String outputContentClaimContainer) { + this.outputContentClaimContainer = outputContentClaimContainer; + } + + /** + * Returns the Identifier of the output Content Claim, or null + * if no Content Claim exists. + * + * @return + */ + public String getOutputContentClaimIdentifier() { + return outputContentClaimIdentifier; + } + + public void setOutputContentClaimIdentifier(String outputContentClaimIdentifier) { + this.outputContentClaimIdentifier = outputContentClaimIdentifier; + } + + /** + * Returns the offset into the the output Content Claim where the FlowFile's + * content begins, or null if no Content Claim exists. + * + * @return + */ + public Long getOutputContentClaimOffset() { + return outputContentClaimOffset; + } + + public void setOutputContentClaimOffset(Long outputContentClaimOffset) { + this.outputContentClaimOffset = outputContentClaimOffset; + } + + /** + * Returns the formatted file size of the input content claim. + * + * @return + */ + public String getOutputContentClaimFileSize() { + return outputContentClaimFileSize; + } + + public void setOutputContentClaimFileSize(String outputContentClaimFileSize) { + this.outputContentClaimFileSize = outputContentClaimFileSize; + } + + /** + * Returns the number of bytes of the input content claim. + * + * @return + */ + public Long getOutputContentClaimFileSizeBytes() { + return outputContentClaimFileSizeBytes; + } + + public void setOutputContentClaimFileSizeBytes(Long outputContentClaimFileSizeBytes) { + this.outputContentClaimFileSizeBytes = outputContentClaimFileSizeBytes; + } + + /** + * Returns whether or not the input content is still available. + * + * @return + */ + public Boolean getInputContentAvailable() { + return inputContentAvailable; + } + + public void setInputContentAvailable(Boolean inputContentAvailable) { + this.inputContentAvailable = inputContentAvailable; + } + + /** + * Returns the Section in which the input Content Claim lives, or + * null if no Content Claim exists. + * + * @return + */ + public String getInputContentClaimSection() { + return inputContentClaimSection; + } + + public void setInputContentClaimSection(String inputContentClaimSection) { + this.inputContentClaimSection = inputContentClaimSection; + } + + /** + * Returns the Container in which the input Content Claim lives, or + * null if no Content Claim exists. + * + * @return + */ + public String getInputContentClaimContainer() { + return inputContentClaimContainer; + } + + public void setInputContentClaimContainer(String inputContentClaimContainer) { + this.inputContentClaimContainer = inputContentClaimContainer; + } + + /** + * Returns the Identifier of the input Content Claim, or null + * if no Content Claim exists. + * + * @return + */ + public String getInputContentClaimIdentifier() { + return inputContentClaimIdentifier; + } + + public void setInputContentClaimIdentifier(String inputContentClaimIdentifier) { + this.inputContentClaimIdentifier = inputContentClaimIdentifier; + } + + /** + * Returns the offset into the the input Content Claim where the FlowFile's + * content begins, or null if no Content Claim exists. + * + * @return + */ + public Long getInputContentClaimOffset() { + return inputContentClaimOffset; + } + + public void setInputContentClaimOffset(Long inputContentClaimOffset) { + this.inputContentClaimOffset = inputContentClaimOffset; + } + + /** + * Returns the formatted file size of the input content claim. + * + * @return + */ + public String getInputContentClaimFileSize() { + return inputContentClaimFileSize; + } + + public void setInputContentClaimFileSize(String inputContentClaimFileSize) { + this.inputContentClaimFileSize = inputContentClaimFileSize; + } + + /** + * Returns the number of bytes of the input content claim. + * + * @return + */ + public Long getInputContentClaimFileSizeBytes() { + return inputContentClaimFileSizeBytes; + } + + public void setInputContentClaimFileSizeBytes(Long inputContentClaimFileSizeBytes) { + this.inputContentClaimFileSizeBytes = inputContentClaimFileSizeBytes; + } + + /** + * Returns whether or not replay is available. + * + * @return + */ + public Boolean getReplayAvailable() { + return replayAvailable; + } + + public void setReplayAvailable(Boolean replayAvailable) { + this.replayAvailable = replayAvailable; + } + + /** + * Returns the explanation as to why replay is unavailable. + * + * @return + */ + public String getReplayExplanation() { + return replayExplanation; + } + + public void setReplayExplanation(String replayExplanation) { + this.replayExplanation = replayExplanation; + } + + /** + * Returns identifier of the FlowFile Queue / Connection from which the + * FlowFile was pulled to generate this event, or null if + * either the queue is unknown or the FlowFile was created by this event. + * + * @return + */ + public String getSourceConnectionIdentifier() { + return sourceConnectionIdentifier; + } + + public void setSourceConnectionIdentifier(String sourceConnectionIdentifier) { + this.sourceConnectionIdentifier = sourceConnectionIdentifier; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java new file mode 100644 index 0000000000..7f939b9d15 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceOptionsDTO.java @@ -0,0 +1,43 @@ +/* + * 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.web.api.dto.provenance; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * A provenance search options. + */ +@XmlType(name = "provenanceOptions") +public class ProvenanceOptionsDTO { + + private List searchableFields; + + /** + * The available searchable fields for this NiFi instance. + * + * @return + */ + public List getSearchableFields() { + return searchableFields; + } + + public void setSearchableFields(List searchableFields) { + this.searchableFields = searchableFields; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java new file mode 100644 index 0000000000..e9e54534ed --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceRequestDTO.java @@ -0,0 +1,118 @@ +/* + * 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.web.api.dto.provenance; + +import java.util.Date; +import java.util.Map; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; + +/** + * A request for provenance. + */ +@XmlType(name = "provenanceRequest") +public class ProvenanceRequestDTO { + + private Map searchTerms; + private Date startDate; + private Date endDate; + private String minimumFileSize; + private String maximumFileSize; + private Integer maxResults; + + /** + * Returns the search terms to use for this search + * + * @return + */ + public Map getSearchTerms() { + return searchTerms; + } + + public void setSearchTerms(final Map searchTerms) { + this.searchTerms = searchTerms; + } + + /** + * The earliest event time to include in the query + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getStartDate() { + return startDate; + } + + public void setStartDate(Date startDate) { + this.startDate = startDate; + } + + /** + * The latest event time to include in the query + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getEndDate() { + return endDate; + } + + public void setEndDate(Date endDate) { + this.endDate = endDate; + } + + /** + * The minimum file size to include in the query. + * + * @return + */ + public String getMinimumFileSize() { + return minimumFileSize; + } + + public void setMinimumFileSize(String minimumFileSize) { + this.minimumFileSize = minimumFileSize; + } + + /** + * The maximum file size to include in the query. + * + * @return + */ + public String getMaximumFileSize() { + return maximumFileSize; + } + + public void setMaximumFileSize(String maximumFileSize) { + this.maximumFileSize = maximumFileSize; + } + + /** + * The number of max results. + * + * @return + */ + public Integer getMaxResults() { + return maxResults; + } + + public void setMaxResults(Integer maxResults) { + this.maxResults = maxResults; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java new file mode 100644 index 0000000000..d6ea8a00d7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceResultsDTO.java @@ -0,0 +1,136 @@ +/* + * 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.web.api.dto.provenance; + +import org.apache.nifi.web.api.dto.util.DateTimeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import java.util.Date; +import java.util.List; +import java.util.Set; + +/** + * Results of a provenance request. + */ +@XmlType(name = "provenanceResults") +public class ProvenanceResultsDTO { + + private List provenanceEvents; + private String total; + private Long totalCount; + private Date generated; + private Date oldestEvent; + private Integer timeOffset; + + private Set errors; + + /** + * Any error messages. + * + * @return + */ + public Set getErrors() { + return errors; + } + + public void setErrors(Set errors) { + this.errors = errors; + } + + /** + * The provenance events that matched the search criteria. + * + * @return + */ + public List getProvenanceEvents() { + return provenanceEvents; + } + + public void setProvenanceEvents(List provenanceEvents) { + this.provenanceEvents = provenanceEvents; + } + + /** + * The total number of results formatted. + * + * @return + */ + public String getTotal() { + return total; + } + + public void setTotal(String total) { + this.total = total; + } + + /** + * The total number of results. + * + * @return + */ + public Long getTotalCount() { + return totalCount; + } + + public void setTotalCount(Long totalCount) { + this.totalCount = totalCount; + } + + /** + * When the search was performed. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } + + /** + * The oldest event available in the provenance repository. + * + * @return + */ + @XmlJavaTypeAdapter(DateTimeAdapter.class) + public Date getOldestEvent() { + return oldestEvent; + } + + public void setOldestEvent(Date oldestEvent) { + this.oldestEvent = oldestEvent; + } + + /** + * The time offset on the server thats used for event time. + * + * @return + */ + public Integer getTimeOffset() { + return timeOffset; + } + + public void setTimeOffset(Integer timeOffset) { + this.timeOffset = timeOffset; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java new file mode 100644 index 0000000000..af89f97c3f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/ProvenanceSearchableFieldDTO.java @@ -0,0 +1,84 @@ +/* + * 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.web.api.dto.provenance; + +import javax.xml.bind.annotation.XmlType; + +/** + * A searchable field for provenance queries. + */ +@XmlType(name = "provenanceSearchableField") +public class ProvenanceSearchableFieldDTO { + + private String id; + private String field; + private String label; + private String type; + + /** + * The id of this searchable field. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The field. + * + * @return + */ + public String getField() { + return field; + } + + public void setField(String field) { + this.field = field; + } + + /** + * The label for this field. + * + * @return + */ + public String getLabel() { + return label; + } + + public void setLabel(String label) { + this.label = label; + } + + /** + * The type for this field. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java new file mode 100644 index 0000000000..745105ed01 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageDTO.java @@ -0,0 +1,161 @@ +/* + * 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.web.api.dto.provenance.lineage; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimestampAdapter; + +/** + * Represents the lineage for a flowfile. + */ +@XmlType(name = "lineage") +public class LineageDTO { + + private String id; + private String uri; + private String clusterNodeId; + + private Date submissionTime; + private Date expiration; + private Integer percentCompleted; + private Boolean finished; + + private LineageRequestDTO request; + private LineageResultsDTO results; + + /** + * The id of this lineage. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The uri for this lineage. + * + * @return + */ + public String getUri() { + return uri; + } + + public void setUri(String uri) { + this.uri = uri; + } + + /** + * The id of the node in the cluster where this lineage originated. + * + * @return + */ + public String getClusterNodeId() { + return clusterNodeId; + } + + public void setClusterNodeId(String clusterNodeId) { + this.clusterNodeId = clusterNodeId; + } + + /** + * The submission time for this lineage. + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getSubmissionTime() { + return submissionTime; + } + + public void setSubmissionTime(Date submissionTime) { + this.submissionTime = submissionTime; + } + + /** + * The expiration of this lineage. + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getExpiration() { + return expiration; + } + + public void setExpiration(Date expiration) { + this.expiration = expiration; + } + + /** + * Percent completed for this result. + * + * @return + */ + public Integer getPercentCompleted() { + return percentCompleted; + } + + public void setPercentCompleted(Integer percentCompleted) { + this.percentCompleted = percentCompleted; + } + + /** + * Whether or not the request is finished running. + * + * @return + */ + public Boolean getFinished() { + return finished; + } + + public void setFinished(Boolean finished) { + this.finished = finished; + } + + /** + * The lineage request. + * + * @return + */ + public LineageRequestDTO getRequest() { + return request; + } + + public void setRequest(LineageRequestDTO request) { + this.request = request; + } + + /** + * The results of this lineage. + * + * @return + */ + public LineageResultsDTO getResults() { + return results; + } + + public void setResults(LineageResultsDTO results) { + this.results = results; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java new file mode 100644 index 0000000000..23cf892cfa --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageRequestDTO.java @@ -0,0 +1,88 @@ +/* + * 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.web.api.dto.provenance.lineage; + +import javax.xml.bind.annotation.XmlEnum; +import javax.xml.bind.annotation.XmlType; + +/** + * Represents the request for lineage for a flowfile. + */ +@XmlType(name = "lineageRequest") +public class LineageRequestDTO { + + /** + * The type of this lineage request. + */ + @XmlType(name = "lineageRequestType") + @XmlEnum + public enum LineageRequestType { + + PARENTS, + CHILDREN, + FLOWFILE; + }; + + private Long eventId; + private LineageRequestType lineageRequestType; + + private String uuid; + + /** + * The event id that was used to generate this lineage. + * + * @return + */ + public Long getEventId() { + return eventId; + } + + public void setEventId(Long eventId) { + this.eventId = eventId; + } + + /** + * The type of lineage request. Either 'PARENTS', 'CHILDREN', or 'FLOWFILE'. + * PARENTS will return the lineage for the flowfiles that are parents of the + * specified event. CHILDREN will return the lineage of for the flowfiles + * that are children of the specified event. FLOWFILE will return the + * lineage for the specified flowfile. + * + * @return + */ + public LineageRequestType getLineageRequestType() { + return lineageRequestType; + } + + public void setLineageRequestType(LineageRequestType lineageRequestType) { + this.lineageRequestType = lineageRequestType; + } + + /** + * The uuid that was used to generate this lineage. + * + * @return + */ + public String getUuid() { + return uuid; + } + + public void setUuid(String uuid) { + this.uuid = uuid; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java new file mode 100644 index 0000000000..77b6e7ace7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/LineageResultsDTO.java @@ -0,0 +1,73 @@ +/* + * 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.web.api.dto.provenance.lineage; + +import java.util.List; +import java.util.Set; +import javax.xml.bind.annotation.XmlType; + +/** + * Represents the lineage results. + */ +@XmlType(name = "lineage") +public class LineageResultsDTO { + + private Set errors; + + private List nodes; + private List links; + + /** + * Any error messages. + * + * @return + */ + public Set getErrors() { + return errors; + } + + public void setErrors(Set errors) { + this.errors = errors; + } + + /** + * The nodes. + * + * @return + */ + public List getNodes() { + return nodes; + } + + public void setNodes(List nodes) { + this.nodes = nodes; + } + + /** + * The links. + * + * @return + */ + public List getLinks() { + return links; + } + + public void setLinks(List links) { + this.links = links; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java new file mode 100644 index 0000000000..f3bbcefbed --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceLinkDTO.java @@ -0,0 +1,101 @@ +/* + * 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.web.api.dto.provenance.lineage; + +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimestampAdapter; + +/** + * A link between an event or flowfile within a provenance lineage. + */ +@XmlType(name = "provenanceLink") +public class ProvenanceLinkDTO { + + private String sourceId; + private String targetId; + private String flowFileUuid; + private Date timestamp; + private Long millis; + + /** + * The source node id. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The target node id. + * + * @return + */ + public String getTargetId() { + return targetId; + } + + public void setTargetId(String targetId) { + this.targetId = targetId; + } + + /** + * The flowfile uuid that traversed this link. + * + * @return + */ + public String getFlowFileUuid() { + return flowFileUuid; + } + + public void setFlowFileUuid(String flowFileUuid) { + this.flowFileUuid = flowFileUuid; + } + + /** + * The timestamp of this link (based on the destination). + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + /** + * The number of millis since epoch. + * + * @return + */ + public Long getMillis() { + return millis; + } + + public void setMillis(Long millis) { + this.millis = millis; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java new file mode 100644 index 0000000000..c15c598cd0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/provenance/lineage/ProvenanceNodeDTO.java @@ -0,0 +1,162 @@ +/* + * 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.web.api.dto.provenance.lineage; + +import java.util.Date; +import java.util.List; + +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.web.api.dto.util.TimestampAdapter; + +/** + * A node within a provenance lineage. May represent either an event or a + * flowfile. + */ +@XmlType(name = "provenanceNode") +public class ProvenanceNodeDTO { + + private String id; + private String flowFileUuid; + private List parentUuids; + private List childUuids; + private String clusterNodeIdentifier; + private String type; + private String eventType; + private Long millis; + private Date timestamp; + + /** + * The id of the node. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The flowfile uuid for this provenance event. + * + * @return + */ + public String getFlowFileUuid() { + return flowFileUuid; + } + + public void setFlowFileUuid(String flowFileUuid) { + this.flowFileUuid = flowFileUuid; + } + + /** + * The parent flowfile uuids for this provenance event. + * + * @return + */ + public List getParentUuids() { + return parentUuids; + } + + public void setParentUuids(List parentUuids) { + this.parentUuids = parentUuids; + } + + /** + * The child flowfile uuids for this provenance event. + * + * @return + */ + public List getChildUuids() { + return childUuids; + } + + public void setChildUuids(List childUuids) { + this.childUuids = childUuids; + } + + /** + * The node identifier that this event/flowfile originated from. + * + * @return + */ + public String getClusterNodeIdentifier() { + return clusterNodeIdentifier; + } + + public void setClusterNodeIdentifier(String clusterNodeIdentifier) { + this.clusterNodeIdentifier = clusterNodeIdentifier; + } + + /** + * The type of node. + * + * @return + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * If this is an event node, this is the type of event. + * + * @return + */ + public String getEventType() { + return eventType; + } + + public void setEventType(String eventType) { + this.eventType = eventType; + } + + /** + * The timestamp of this node. + * + * @return + */ + @XmlJavaTypeAdapter(TimestampAdapter.class) + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + /** + * The number of millis since epoch. + * + * @return + */ + public Long getMillis() { + return millis; + } + + public void setMillis(Long millis) { + this.millis = millis; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java new file mode 100644 index 0000000000..83d7a91319 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/ComponentSearchResultDTO.java @@ -0,0 +1,85 @@ +/* + * 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.web.api.dto.search; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * The components that match a search performed on this NiFi. + */ +@XmlType(name = "componentSearchResult") +public class ComponentSearchResultDTO { + + private String id; + private String groupId; + private String name; + private List matches; + + /** + * The id of the component that matched. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The group id of the component that matched. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The name of the component that matched. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * What matched the search string for this component. + * + * @return + */ + public List getMatches() { + return matches; + } + + public void setMatches(List matches) { + this.matches = matches; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java new file mode 100644 index 0000000000..7c250ee806 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/NodeSearchResultDTO.java @@ -0,0 +1,56 @@ +/* + * 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.web.api.dto.search; + +import javax.xml.bind.annotation.XmlType; + +/** + * A search result for a matching node. + */ +@XmlType(name = "nodeSearchResult") +public class NodeSearchResultDTO { + + private String id; + private String address; + + /** + * The id of the node that was matched. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The address of the node that was matched. + * + * @return + */ + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java new file mode 100644 index 0000000000..212d4f2ef3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java @@ -0,0 +1,128 @@ +/* + * 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.web.api.dto.search; + +import java.util.ArrayList; +import java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * The results of a search on this NiFi. + */ +@XmlType(name = "searchResults") +public class SearchResultsDTO { + + private List processorResults = new ArrayList<>(); + private List connectionResults = new ArrayList<>(); + private List processGroupResults = new ArrayList<>(); + private List inputPortResults = new ArrayList<>(); + private List outputPortResults = new ArrayList<>(); + private List remoteProcessGroupResults = new ArrayList<>(); + private List funnelResults = new ArrayList<>(); + + /** + * The processors that matched the search. + * + * @return + */ + public List getProcessorResults() { + return processorResults; + } + + public void setProcessorResults(List processorResults) { + this.processorResults = processorResults; + } + + /** + * The connections that matched the search. + * + * @return + */ + public List getConnectionResults() { + return connectionResults; + } + + public void setConnectionResults(List connectionResults) { + this.connectionResults = connectionResults; + } + + /** + * The process group that matched the search. + * + * @return + */ + public List getProcessGroupResults() { + return processGroupResults; + } + + public void setProcessGroupResults(List processGroupResults) { + this.processGroupResults = processGroupResults; + } + + /** + * The input ports that matched the search. + * + * @return + */ + public List getInputPortResults() { + return inputPortResults; + } + + /** + * The output ports that matched the search. + * + * @return + */ + public List getOutputPortResults() { + return outputPortResults; + } + + public void setInputPortResults(List inputPortResults) { + this.inputPortResults = inputPortResults; + } + + public void setOutputPortResults(List outputPortResults) { + this.outputPortResults = outputPortResults; + } + + /** + * The remote process groups that matched the search. + * + * @return + */ + public List getRemoteProcessGroupResults() { + return remoteProcessGroupResults; + } + + public void setRemoteProcessGroupResults(List remoteProcessGroupResults) { + this.remoteProcessGroupResults = remoteProcessGroupResults; + } + + /** + * The funnels that matched the search. + * + * @return + */ + public List getFunnelResults() { + return funnelResults; + } + + public void setFunnelResults(List funnelResults) { + this.funnelResults = funnelResults; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java new file mode 100644 index 0000000000..ab5636d680 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserGroupSearchResultDTO.java @@ -0,0 +1,42 @@ +/* + * 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.web.api.dto.search; + +import javax.xml.bind.annotation.XmlType; + +/** + * A search result for a matching user group. + */ +@XmlType(name = "userGroupSearchResult") +public class UserGroupSearchResultDTO { + + private String group; + + /** + * The name of the group that matched. + * + * @return + */ + public String getGroup() { + return group; + } + + public void setGroup(String group) { + this.group = group; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java new file mode 100644 index 0000000000..9402eeb0d9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/search/UserSearchResultDTO.java @@ -0,0 +1,56 @@ +/* + * 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.web.api.dto.search; + +import javax.xml.bind.annotation.XmlType; + +/** + * A search result for a matching user. + */ +@XmlType(name = "userSearchResult") +public class UserSearchResultDTO { + + private String userName; + private String userDn; + + /** + * The dn of the user that matched. + * + * @return + */ + public String getUserDn() { + return userDn; + } + + public void setUserDn(String userDn) { + this.userDn = userDn; + } + + /** + * The username of user that matched. + * + * @return + */ + public String getUserName() { + return userName; + } + + public void setUserName(String userName) { + this.userName = userName; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java new file mode 100644 index 0000000000..f0e11d6398 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterConnectionStatusDTO.java @@ -0,0 +1,89 @@ +/* + * 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.web.api.dto.status; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * DTO for serializing the a connection's status across the cluster. + */ +@XmlType(name = "clusterConnectionStatus") +public class ClusterConnectionStatusDTO { + + private Collection nodeConnectionStatus; + private Date statsLastRefreshed; + private String connectionId; + private String connectionName; + + /** + * The time the status were last refreshed. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + + /** + * The connection id. + * + * @return + */ + public String getConnectionId() { + return connectionId; + } + + public void setConnectionId(String connectionId) { + this.connectionId = connectionId; + } + + /** + * The connection name. + * + * @return + */ + public String getConnectionName() { + return connectionName; + } + + public void setConnectionName(String connectionName) { + this.connectionName = connectionName; + } + + /** + * Collection of node connection status DTO. + * + * @return The collection of node connection status DTO + */ + public Collection getNodeConnectionStatus() { + return nodeConnectionStatus; + } + + public void setNodeConnectionStatus(Collection nodeConnectionStatus) { + this.nodeConnectionStatus = nodeConnectionStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java new file mode 100644 index 0000000000..4d7fa4d611 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterPortStatusDTO.java @@ -0,0 +1,89 @@ +/* + * 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.web.api.dto.status; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * DTO for serializing the a port's status across the cluster. + */ +@XmlType(name = "clusterPortStatus") +public class ClusterPortStatusDTO { + + private Collection nodePortStatus; + private Date statsLastRefreshed; + private String portId; + private String portName; + + /** + * The time the status was last refreshed. + * + * @return the time the status were last refreshed + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + + /** + * The port status from each node in the cluster. + * + * @return + */ + public Collection getNodePortStatus() { + return nodePortStatus; + } + + public void setNodePortStatus(Collection nodePortStatus) { + this.nodePortStatus = nodePortStatus; + } + + /** + * The port id. + * + * @return + */ + public String getPortId() { + return portId; + } + + public void setPortId(String portId) { + this.portId = portId; + } + + /** + * The port name. + * + * @return + */ + public String getPortName() { + return portName; + } + + public void setPortName(String portName) { + this.portName = portName; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java new file mode 100644 index 0000000000..b9f45f2b85 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterProcessorStatusDTO.java @@ -0,0 +1,117 @@ +/* + * 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.web.api.dto.status; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * DTO for serializing the a processor's status across the cluster. + */ +@XmlType(name = "clusterProcessorStatus") +public class ClusterProcessorStatusDTO { + + private Collection nodeProcessorStatus; + private Date statsLastRefreshed; + private String processorId; + private String processorName; + private String processorType; + private String processorRunStatus; + + /** + * The time the status were last refreshed. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + + /** + * The processor id. + * + * @return + */ + public String getProcessorId() { + return processorId; + } + + public void setProcessorId(String processorId) { + this.processorId = processorId; + } + + /** + * The processor name. + * + * @return + */ + public String getProcessorName() { + return processorName; + } + + public void setProcessorName(String processorName) { + this.processorName = processorName; + } + + /** + * The processor type. + * + * @return + */ + public String getProcessorType() { + return processorType; + } + + public void setProcessorType(String processorType) { + this.processorType = processorType; + } + + /** + * The processor run status. + * + * @return + */ + public String getProcessorRunStatus() { + return processorRunStatus; + } + + public void setProcessorRunStatus(String runStatus) { + this.processorRunStatus = runStatus; + } + + /** + * Collection of node processor status DTO. + * + * @return The collection of node processor status DTO + */ + public Collection getNodeProcessorStatus() { + return nodeProcessorStatus; + } + + public void setNodeProcessorStatus(Collection nodeProcessorStatus) { + this.nodeProcessorStatus = nodeProcessorStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java new file mode 100644 index 0000000000..d6b26d414e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterRemoteProcessGroupStatusDTO.java @@ -0,0 +1,89 @@ +/* + * 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.web.api.dto.status; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * DTO for serializing the a remote process group's status across the cluster. + */ +@XmlType(name = "clusterRemoteProcessGroupStatus") +public class ClusterRemoteProcessGroupStatusDTO { + + private Collection nodeRemoteProcessGroupStatus; + private Date statsLastRefreshed; + private String remoteProcessGroupId; + private String remoteProcessGroupName; + + /** + * The time the status was last refreshed. + * + * @return the time the status were last refreshed + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + + /** + * The remote process group status from each node in the cluster. + * + * @return + */ + public Collection getNodeRemoteProcessGroupStatus() { + return nodeRemoteProcessGroupStatus; + } + + public void setNodeRemoteProcessGroupStatus(Collection nodeRemoteProcessGroupStatus) { + this.nodeRemoteProcessGroupStatus = nodeRemoteProcessGroupStatus; + } + + /** + * The remote process group id. + * + * @return + */ + public String getRemoteProcessGroupId() { + return remoteProcessGroupId; + } + + public void setRemoteProcessGroupId(String remoteProcessGroupId) { + this.remoteProcessGroupId = remoteProcessGroupId; + } + + /** + * The remote process group name. + * + * @return + */ + public String getRemoteProcessGroupName() { + return remoteProcessGroupName; + } + + public void setRemoteProcessGroupName(String remoteProcessGroupName) { + this.remoteProcessGroupName = remoteProcessGroupName; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java new file mode 100644 index 0000000000..d83399174b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusDTO.java @@ -0,0 +1,44 @@ +/* + * 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.web.api.dto.status; + +import java.util.Collection; + +import javax.xml.bind.annotation.XmlType; + +/** + * DTO for serializing the controller status. + */ +@XmlType(name = "clusterStatus") +public class ClusterStatusDTO { + + private Collection nodeStatus; + + /** + * The collection of the node status DTOs. + * + * @return + */ + public Collection getNodeStatus() { + return nodeStatus; + } + + public void setNodeStatus(Collection nodeStatus) { + this.nodeStatus = nodeStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.java new file mode 100644 index 0000000000..31a994845d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ClusterStatusHistoryDTO.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.web.api.dto.status; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * DTO for serializing the a status history across the cluster. + */ +@XmlType(name = "clusterStatusHistory") +public class ClusterStatusHistoryDTO { + + private Collection nodeStatusHistory; + private StatusHistoryDTO clusterStatusHistory; + private Date generated; + + /** + * When this status history was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } + + /** + * The status history from each node in the cluster. + * + * @return + */ + public Collection getNodeStatusHistory() { + return nodeStatusHistory; + } + + public void setNodeStatusHistory(Collection nodeStatusHistory) { + this.nodeStatusHistory = nodeStatusHistory; + } + + /** + * The status history for this component across the entire cluster. + * + * @return + */ + public StatusHistoryDTO getClusterStatusHistory() { + return clusterStatusHistory; + } + + public void setClusterStatusHistory(StatusHistoryDTO clusterStatusHistory) { + this.clusterStatusHistory = clusterStatusHistory; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java new file mode 100644 index 0000000000..387cce5c89 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ConnectionStatusDTO.java @@ -0,0 +1,198 @@ +/* + * 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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; + +/** + * DTO for serializing the status of a connection. + */ +@XmlType(name = "connectionStatus") +public class ConnectionStatusDTO { + + private String id; + private String groupId; + private String name; + private String input; + private String queuedCount; + private String queuedSize; + private String queued; + private String output; + + private String sourceId; + private String sourceName; + private String destinationId; + private String destinationName; + + /* getters / setters */ + /** + * The id for the connection. + * + * @return The connection id + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The ID of the Process Group to which this processor belongs. + * + * @return the ID of the Process Group to which this processor belongs. + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(final String groupId) { + this.groupId = groupId; + } + + /** + * The name of this connection. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The total count of flow files that are queued. + * + * @return + */ + public String getQueuedCount() { + return queuedCount; + } + + public void setQueuedCount(String queuedCount) { + this.queuedCount = queuedCount; + } + + /** + * The total size of flow files that are queued. + * + * @return + */ + public String getQueuedSize() { + return queuedSize; + } + + public void setQueuedSize(String queuedSize) { + this.queuedSize = queuedSize; + } + + /** + * The total count and size of flow files that are queued. + * + * @return The total count and size of queued flow files + */ + public String getQueued() { + return queued; + } + + public void setQueued(String queued) { + this.queued = queued; + } + + /** + * The id of the source of this connection. + * + * @return + */ + public String getSourceId() { + return sourceId; + } + + public void setSourceId(String sourceId) { + this.sourceId = sourceId; + } + + /** + * The name of the source of this connection. + * + * @return + */ + public String getSourceName() { + return sourceName; + } + + public void setSourceName(String sourceName) { + this.sourceName = sourceName; + } + + /** + * The id of the destination of this connection. + * + * @return + */ + public String getDestinationId() { + return destinationId; + } + + public void setDestinationId(String destinationId) { + this.destinationId = destinationId; + } + + /** + * The name of the destination of this connection. + * + * @return + */ + public String getDestinationName() { + return destinationName; + } + + public void setDestinationName(String destinationName) { + this.destinationName = destinationName; + } + + /** + * The input for this connection. + * + * @return + */ + public String getInput() { + return input; + } + + public void setInput(String input) { + this.input = input; + } + + /** + * The output for this connection. + * + * @return + */ + public String getOutput() { + return output; + } + + public void setOutput(String output) { + this.output = output; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java new file mode 100644 index 0000000000..9e063e44b8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ControllerStatusDTO.java @@ -0,0 +1,187 @@ +/* + * 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.web.api.dto.status; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.BulletinDTO; + +/** + * The status of this NiFi controller. + */ +@XmlType(name = "controllerStatus") +public class ControllerStatusDTO { + + private Integer activeThreadCount; + private String queued; + private String connectedNodes; + private Boolean hasPendingAccounts; + + private Integer runningCount; + private Integer stoppedCount; + private Integer invalidCount; + private Integer disabledCount; + private Integer activeRemotePortCount; + private Integer inactiveRemotePortCount; + + private List bulletins; + + /** + * The active thread count. + * + * @return The active thread count + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * The queue for the controller. + * + * @return + */ + public String getQueued() { + return queued; + } + + public void setQueued(String queued) { + this.queued = queued; + } + + /** + * Used in clustering, will report the number of nodes connected vs the + * number of nodes in the cluster. + * + * @return + */ + public String getConnectedNodes() { + return connectedNodes; + } + + public void setConnectedNodes(String connectedNodes) { + this.connectedNodes = connectedNodes; + } + + /** + * System bulletins to be reported to the user. + * + * @return + */ + public List getBulletins() { + return bulletins; + } + + public void setBulletins(List bulletins) { + this.bulletins = bulletins; + } + + /** + * Whether or not there are pending user requests. + * + * @return + */ + public Boolean getHasPendingAccounts() { + return hasPendingAccounts; + } + + public void setHasPendingAccounts(Boolean hasPendingAccounts) { + this.hasPendingAccounts = hasPendingAccounts; + } + + /** + * The number of running components in this controller. + * + * @return + */ + public Integer getRunningCount() { + return runningCount; + } + + public void setRunningCount(Integer runningCount) { + this.runningCount = runningCount; + } + + /** + * The number of stopped components in this controller. + * + * @return + */ + public Integer getStoppedCount() { + return stoppedCount; + } + + public void setStoppedCount(Integer stoppedCount) { + this.stoppedCount = stoppedCount; + } + + /** + * The number of invalid components in this controller. + * + * @return + */ + public Integer getInvalidCount() { + return invalidCount; + } + + public void setInvalidCount(Integer invalidCount) { + this.invalidCount = invalidCount; + } + + /** + * The number of disabled components in this controller. + * + * @return + */ + public Integer getDisabledCount() { + return disabledCount; + } + + public void setDisabledCount(Integer disabledCount) { + this.disabledCount = disabledCount; + } + + /** + * The number of active remote ports in this controller. + * + * @return + */ + public Integer getActiveRemotePortCount() { + return activeRemotePortCount; + } + + public void setActiveRemotePortCount(Integer activeRemotePortCount) { + this.activeRemotePortCount = activeRemotePortCount; + } + + /** + * The number of inactive remote ports in this controller. + * + * @return + */ + public Integer getInactiveRemotePortCount() { + return inactiveRemotePortCount; + } + + public void setInactiveRemotePortCount(Integer inactiveRemotePortCount) { + this.inactiveRemotePortCount = inactiveRemotePortCount; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.java new file mode 100644 index 0000000000..f7cba448a2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeConnectionStatusDTO.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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * DTO for serializing the connection status for a particular node. + */ +@XmlType(name = "nodeConnectionStatus") +public class NodeConnectionStatusDTO { + + private NodeDTO node; + private ConnectionStatusDTO connectionStatus; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The connection's status. + * + * @return + */ + public ConnectionStatusDTO getConnectionStatus() { + return connectionStatus; + } + + public void setConnectionStatus(ConnectionStatusDTO connectionStatus) { + this.connectionStatus = connectionStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.java new file mode 100644 index 0000000000..93d89ac1e2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodePortStatusDTO.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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * DTO for serializing the port status for a particular node. + */ +@XmlType(name = "nodePortStatus") +public class NodePortStatusDTO { + + private NodeDTO node; + private PortStatusDTO portStatus; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The port's status. + * + * @return + */ + public PortStatusDTO getPortStatus() { + return portStatus; + } + + public void setPortStatus(PortStatusDTO portStatus) { + this.portStatus = portStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.java new file mode 100644 index 0000000000..afff4b10f5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeProcessorStatusDTO.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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * DTO for serializing the processor status for a particular node. + */ +@XmlType(name = "nodeProcessorStatus") +public class NodeProcessorStatusDTO { + + private NodeDTO node; + private ProcessorStatusDTO processorStatus; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The processor's status. + * + * @return + */ + public ProcessorStatusDTO getProcessorStatus() { + return processorStatus; + } + + public void setProcessorStatus(ProcessorStatusDTO processorStatus) { + this.processorStatus = processorStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.java new file mode 100644 index 0000000000..03923bf35b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeRemoteProcessGroupStatusDTO.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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * DTO for serializing the remote process group status for a particular node. + */ +@XmlType(name = "nodeRemoteProcessGroupStatus") +public class NodeRemoteProcessGroupStatusDTO { + + private NodeDTO node; + private RemoteProcessGroupStatusDTO remoteProcessGroupStatus; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The remote process group's status. + * + * @return + */ + public RemoteProcessGroupStatusDTO getRemoteProcessGroupStatus() { + return remoteProcessGroupStatus; + } + + public void setRemoteProcessGroupStatus(RemoteProcessGroupStatusDTO remoteProcessGroupStatus) { + this.remoteProcessGroupStatus = remoteProcessGroupStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.java new file mode 100644 index 0000000000..03647cd173 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusDTO.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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * DTO for serializing the node status. + */ +@XmlType(name = "nodeStatus") +public class NodeStatusDTO { + + private NodeDTO node; + private ProcessGroupStatusDTO controllerStatus; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The controller status. + * + * @return + */ + public ProcessGroupStatusDTO getControllerStatus() { + return controllerStatus; + } + + public void setControllerStatus(ProcessGroupStatusDTO controllerStatus) { + this.controllerStatus = controllerStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.java new file mode 100644 index 0000000000..c8b05f3e82 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/NodeStatusHistoryDTO.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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * DTO for serializing the status history for a particular node. + */ +@XmlType(name = "nodeStatusHistory") +public class NodeStatusHistoryDTO { + + private NodeDTO node; + private StatusHistoryDTO statusHistory; + + /** + * The node. + * + * @return + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + + /** + * The processor status history. + * + * @return + */ + public StatusHistoryDTO getStatusHistory() { + return statusHistory; + } + + public void setStatusHistory(StatusHistoryDTO statusHistory) { + this.statusHistory = statusHistory; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java new file mode 100644 index 0000000000..c32efdb750 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/PortStatusDTO.java @@ -0,0 +1,142 @@ +/* + * 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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; + +/** + * The status for a port in this NiFi. + */ +@XmlType(name = "portStatus") +public class PortStatusDTO extends StatusDTO { + + private String id; + private String groupId; + private String name; + private Integer activeThreadCount; + private String input; + private String output; + private Boolean transmitting; + private String runStatus; + + /** + * Whether this port has incoming or outgoing connections to a remote NiFi. + * + * @return + */ + public Boolean isTransmitting() { + return transmitting; + } + + public void setTransmitting(Boolean transmitting) { + this.transmitting = transmitting; + } + + /** + * The active thread count for this port. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * The id of this port. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The id of the group this port resides in. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The name of this port. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The run status of this port. + * + * @return + */ + public String getRunStatus() { + return runStatus; + } + + public void setRunStatus(String runStatus) { + this.runStatus = runStatus; + } + + /** + * The total count and size of flow files that have been accepted in the + * last five minutes. + * + * @return The total processed + */ + public String getInput() { + return input; + } + + public void setInput(String input) { + this.input = input; + } + + /** + * The total count and size of flow files that have been processed in the + * last five minutes. + * + * @return The total output + */ + public String getOutput() { + return output; + } + + public void setOutput(String output) { + this.output = output; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java new file mode 100644 index 0000000000..4fa2b641eb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessGroupStatusDTO.java @@ -0,0 +1,244 @@ +/* + * 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.web.api.dto.status; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * The status for a process group in this NiFi. + */ +@XmlType(name = "processGroupStatus") +public class ProcessGroupStatusDTO extends StatusDTO { + + private String id; + private String name; + private Collection connectionStatus; + private Collection processorStatus; + private Collection processGroupStatus; + private Collection remoteProcessGroupStatus; + private Collection inputPortStatus; + private Collection outputPortStatus; + + private String input; + private String queued; + private String read; + private String written; + private String output; + private Integer activeThreadCount; + private Date statsLastRefreshed; + + /** + * The id for the process group. + * + * @return The id for the process group + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name of this process group. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The active thread count for this process group. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * The status of all connections in this process group. + * + * @return The status of all connections + */ + public Collection getConnectionStatus() { + return connectionStatus; + } + + public void setConnectionStatus(Collection connectionStatus) { + this.connectionStatus = connectionStatus; + } + + /** + * The status of all process groups in this process group. + * + * @return The status of all process groups + */ + public Collection getProcessGroupStatus() { + return processGroupStatus; + } + + public void setProcessGroupStatus(Collection processGroupStatus) { + this.processGroupStatus = processGroupStatus; + } + + /** + * The status of all remote process groups in this process group. + * + * @return The status of all remote process groups + */ + public Collection getRemoteProcessGroupStatus() { + return remoteProcessGroupStatus; + } + + public void setRemoteProcessGroupStatus(final Collection remoteProcessGroupStatus) { + this.remoteProcessGroupStatus = remoteProcessGroupStatus; + } + + /** + * The status of all processors in this process group. + * + * @return The status of all processors + */ + public Collection getProcessorStatus() { + return processorStatus; + } + + public void setProcessorStatus(Collection processorStatus) { + this.processorStatus = processorStatus; + } + + /** + * The status of all input ports in this process group. + * + * @return The status of all input ports + */ + public Collection getInputPortStatus() { + return inputPortStatus; + } + + public void setInputPortStatus(Collection inputPortStatus) { + this.inputPortStatus = inputPortStatus; + } + + /** + * The status of all output ports in this process group. + * + * @return The status of all output ports + */ + public Collection getOutputPortStatus() { + return outputPortStatus; + } + + public void setOutputPortStatus(Collection outputPortStatus) { + this.outputPortStatus = outputPortStatus; + } + + /** + * The output stats for this process group. + * + * @return The output stats + */ + public String getOutput() { + return output; + } + + public void setOutput(String output) { + this.output = output; + } + + /** + * The queued stats for this process group. + * + * @return The queued stats + */ + public String getQueued() { + return queued; + } + + public void setQueued(String queued) { + this.queued = queued; + } + + /** + * The read stats for this process group. + * + * @return The read stats + */ + public String getRead() { + return read; + } + + public void setRead(String read) { + this.read = read; + } + + /** + * The written stats for this process group. + * + * @return The written stats + */ + public String getWritten() { + return written; + } + + public void setWritten(String written) { + this.written = written; + } + + /** + * The input stats for this process group. + * + * @return The input stats + */ + public String getInput() { + return input; + } + + public void setInput(String input) { + this.input = input; + } + + /** + * When the status for this process group was calculated. + * + * @return The the status was calculated + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getStatsLastRefreshed() { + return statsLastRefreshed; + } + + public void setStatsLastRefreshed(Date statsLastRefreshed) { + this.statsLastRefreshed = statsLastRefreshed; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java new file mode 100644 index 0000000000..334adad4ed --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/ProcessorStatusDTO.java @@ -0,0 +1,203 @@ +/* + * 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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; + +/** + * DTO for serializing the status of a processor. + */ +@XmlType(name = "processorStatus") +public class ProcessorStatusDTO extends StatusDTO { + + private String id; + private String groupId; + private String name; + private String type; + private String runStatus; + + private String read; + private String written; + + private String input; + private String output; + + private String tasks; + private String tasksDuration; + private Integer activeThreadCount; + + /* getters / setters */ + /** + * The id for the processor. + * + * @return The processor id + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name of the processor. + * + * @return The processor name + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The type of the processor. + * + * @return The processor type + */ + public String getType() { + return type; + } + + public void setType(String type) { + this.type = type; + } + + /** + * The run status of this processor. + * + * @return + */ + public String getRunStatus() { + return runStatus; + } + + public void setRunStatus(String runStatus) { + this.runStatus = runStatus; + } + + /** + * The total count and size of flow files that have been accepted in the + * last five minutes. + * + * @return The total processed + */ + public String getInput() { + return input; + } + + public void setInput(String input) { + this.input = input; + } + + /** + * The number of bytes read. + * + * @return + */ + public String getRead() { + return read; + } + + public void setRead(String read) { + this.read = read; + } + + /** + * The number of bytes written. + * + * @return + */ + public String getWritten() { + return written; + } + + public void setWritten(String written) { + this.written = written; + } + + /** + * The ID of the Process Group to which this processor belongs. + * + * @return the ID of the Process Group to which this processor belongs. + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(final String groupId) { + this.groupId = groupId; + } + + /** + * The total count and size of flow files that have been processed in the + * last five minutes. + * + * @return The total output + */ + public String getOutput() { + return output; + } + + public void setOutput(String output) { + this.output = output; + } + + /** + * The number of threads currently running for this Processor + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer threadCount) { + this.activeThreadCount = threadCount; + } + + /** + * The number of task this connectable has had over the last 5 minutes. + * + * @return + */ + public String getTasks() { + return tasks; + } + + public void setTasks(String tasks) { + this.tasks = tasks; + } + + /** + * The total duration of all tasks for this connectable over the last 5 + * minutes. + * + * @return + */ + public String getTasksDuration() { + return tasksDuration; + } + + public void setTasksDuration(String tasksDuration) { + this.tasksDuration = tasksDuration; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java new file mode 100644 index 0000000000..a8fcc9f52d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemotePortStatusDTO.java @@ -0,0 +1,98 @@ +/* + * 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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; + +/** + * The status of a Port on a remote NiFi instance. + */ +@XmlType(name = "remotePortStatus") +public class RemotePortStatusDTO { + + private String id; + private String connectionId; + private String name; + private Boolean running; + private Boolean exists; + + /** + * The id of the connection this remote port is connected to. + * + * @return + */ + public String getConnectionId() { + return connectionId; + } + + public void setConnectionId(String connectionId) { + this.connectionId = connectionId; + } + + /** + * The id of the remote port. + * + * @return + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The name of the remote port. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * Whether or not the remote port exists. + * + * @return + */ + public Boolean getExists() { + return exists; + } + + public void setExists(Boolean exists) { + this.exists = exists; + } + + /** + * Whether or not the remote port is running. + * + * @return + */ + public Boolean getRunning() { + return running; + } + + public void setRunning(Boolean running) { + this.running = running; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java new file mode 100644 index 0000000000..363d4da39b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/RemoteProcessGroupStatusDTO.java @@ -0,0 +1,159 @@ +/* + * 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.web.api.dto.status; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; + +/** + * The status of a remote process group in this NiFi. + */ +@XmlType(name = "remoteProcessGroupStatus") +public class RemoteProcessGroupStatusDTO extends StatusDTO { + + private String id; + private String groupId; + private String name; + private String targetUri; + private String transmissionStatus; + private Integer activeThreadCount; + + private List authorizationIssues; + + private String sent; + private String received; + + /** + * The id for the remote process group. + * + * @return The id for the remote process group + */ + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + /** + * The id of the group this remote process group is in. + * + * @return + */ + public String getGroupId() { + return groupId; + } + + public void setGroupId(String groupId) { + this.groupId = groupId; + } + + /** + * The URI of the target system. + * + * @return + */ + public String getTargetUri() { + return targetUri; + } + + public void setTargetUri(String targetUri) { + this.targetUri = targetUri; + } + + /** + * The name of this remote process group. + * + * @return + */ + public String getName() { + return name; + } + + public void setName(String name) { + this.name = name; + } + + /** + * The transmission status of this remote process group. + * + * @return + */ + public String getTransmissionStatus() { + return transmissionStatus; + } + + public void setTransmissionStatus(String transmissionStatus) { + this.transmissionStatus = transmissionStatus; + } + + /** + * The number of active threads. + * + * @return + */ + public Integer getActiveThreadCount() { + return activeThreadCount; + } + + public void setActiveThreadCount(Integer activeThreadCount) { + this.activeThreadCount = activeThreadCount; + } + + /** + * Returns any remote authorization issues for this remote process group. + * + * @return + */ + public List getAuthorizationIssues() { + return authorizationIssues; + } + + public void setAuthorizationIssues(List authorizationIssues) { + this.authorizationIssues = authorizationIssues; + } + + /** + * Formatted description of the amount of data sent to this remote process + * group. + * + * @return + */ + public String getSent() { + return sent; + } + + public void setSent(String sent) { + this.sent = sent; + } + + /** + * Formatted description of the amount of data received from this remote + * process group. + * + * @return + */ + public String getReceived() { + return received; + } + + public void setReceived(String received) { + this.received = received; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java new file mode 100644 index 0000000000..e4e7a450f9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDTO.java @@ -0,0 +1,43 @@ +/* + * 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.web.api.dto.status; + +import java.util.List; +import javax.xml.bind.annotation.XmlType; +import org.apache.nifi.web.api.dto.BulletinDTO; + +/** + * The status of a component in this NiFi. + */ +@XmlType(name = "status") +public abstract class StatusDTO { + + private List bulletins; + + /** + * Bulletins for this component. + * + * @return + */ + public List getBulletins() { + return bulletins; + } + + public void setBulletins(List bulletins) { + this.bulletins = bulletins; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java new file mode 100644 index 0000000000..a203e10d61 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusDescriptorDTO.java @@ -0,0 +1,101 @@ +/* + * 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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; + +/** + * DTO for serializing a status descriptor. + */ +@XmlType(name = "statusDescriptor") +public class StatusDescriptorDTO { + + public enum Formatter { + + COUNT, + DURATION, + DATA_SIZE + }; + + private String field; + private String label; + private String description; + private String formatter; + + public StatusDescriptorDTO() { + } + + public StatusDescriptorDTO(final String field, final String label, final String description, final String formatter) { + this.field = field; + this.label = label; + this.description = description; + this.formatter = formatter; + } + + /** + * The name of this status field. + * + * @return + */ + public String getField() { + return field; + } + + public void setField(String field) { + this.field = field; + } + + /** + * The label of this status field. + * + * @return + */ + public String getLabel() { + return label; + } + + public void setLabel(String label) { + this.label = label; + } + + /** + * The description of this status field. + * + * @return + */ + public String getDescription() { + return description; + } + + public void setDescription(String description) { + this.description = description; + } + + /** + * The formatter for this descriptor. + * + * @return + */ + public String getFormatter() { + return formatter; + } + + public void setFormatter(String formatter) { + this.formatter = formatter; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java new file mode 100644 index 0000000000..5b97c16d71 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDTO.java @@ -0,0 +1,92 @@ +/* + * 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.web.api.dto.status; + +import java.util.Date; +import java.util.LinkedHashMap; +import java.util.List; +import javax.xml.bind.annotation.XmlType; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * History status for a component in this NiFi. + */ +@XmlType(name = "statusHistory") +public class StatusHistoryDTO { + + private Date generated; + + private LinkedHashMap details; + + private List fieldDescriptors; + private List statusSnapshots; + + /** + * When this status history was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } + + /** + * The component details for this status history. + * + * @return + */ + public LinkedHashMap getDetails() { + return details; + } + + public void setDetails(LinkedHashMap details) { + this.details = details; + } + + /** + * Descriptors for each supported status field. + * + * @return + */ + public List getFieldDescriptors() { + return fieldDescriptors; + } + + public void setFieldDescriptors(List fieldDescriptors) { + this.fieldDescriptors = fieldDescriptors; + } + + /** + * The status snapshots. + * + * @return + */ + public List getStatusSnapshots() { + return statusSnapshots; + } + + public void setStatusSnapshots(List statusSnapshots) { + this.statusSnapshots = statusSnapshots; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java new file mode 100644 index 0000000000..b9d16bba00 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusHistoryDetailDTO.java @@ -0,0 +1,56 @@ +/* + * 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.web.api.dto.status; + +import javax.xml.bind.annotation.XmlType; + +/** + * Detail of a status history metric. + */ +@XmlType(name = "statusHistoryDetail") +public class StatusHistoryDetailDTO { + + private String label; + private String value; + + /** + * The label for this status detail. + * + * @return + */ + public String getLabel() { + return label; + } + + public void setLabel(String label) { + this.label = label; + } + + /** + * The value for this status detail. + * + * @return + */ + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java new file mode 100644 index 0000000000..cc6894a617 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/status/StatusSnapshotDTO.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.dto.status; + +import java.util.Date; +import java.util.Map; +import javax.xml.bind.annotation.XmlType; + +/** + * A snapshot of the status at a given time. + */ +@XmlType(name = "statusSnapshot") +public class StatusSnapshotDTO { + + private Date timestamp; + private Map statusMetrics; + + /** + * The timestamp of this snapshot. + * + * @return + */ + public Date getTimestamp() { + return timestamp; + } + + public void setTimestamp(Date timestamp) { + this.timestamp = timestamp; + } + + /** + * The status metrics. + * + * @return + */ + public Map getStatusMetrics() { + return statusMetrics; + } + + public void setStatusMetrics(Map statusMetrics) { + this.statusMetrics = statusMetrics; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java new file mode 100644 index 0000000000..f860b305e7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/DateTimeAdapter.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.dto.util; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.TimeZone; +import javax.xml.bind.annotation.adapters.XmlAdapter; + +/** + * XmlAdapter for (un)marshalling a date/time. + */ +public class DateTimeAdapter extends XmlAdapter { + + public static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss z"; + + @Override + public String marshal(Date date) throws Exception { + final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT); + formatter.setTimeZone(TimeZone.getDefault()); + return formatter.format(date); + } + + @Override + public Date unmarshal(String date) throws Exception { + final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT); + parser.setTimeZone(TimeZone.getDefault()); + return parser.parse(date); + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java new file mode 100644 index 0000000000..ab77370803 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimeAdapter.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.dto.util; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.TimeZone; +import javax.xml.bind.annotation.adapters.XmlAdapter; + +/** + * XmlAdapter for (un)marshalling a time. + */ +public class TimeAdapter extends XmlAdapter { + + public static final String DEFAULT_TIME_FORMAT = "HH:mm:ss z"; + + @Override + public String marshal(Date date) throws Exception { + final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_TIME_FORMAT); + formatter.setTimeZone(TimeZone.getDefault()); + return formatter.format(date); + } + + @Override + public Date unmarshal(String date) throws Exception { + final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_TIME_FORMAT); + parser.setTimeZone(TimeZone.getDefault()); + return parser.parse(date); + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java new file mode 100644 index 0000000000..49b90a2a13 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/dto/util/TimestampAdapter.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.dto.util; + +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.TimeZone; +import javax.xml.bind.annotation.adapters.XmlAdapter; + +/** + * XmlAdapter for (un)marshalling a date/time. + */ +public class TimestampAdapter extends XmlAdapter { + + public static final String DEFAULT_DATE_TIME_FORMAT = "MM/dd/yyyy HH:mm:ss.SSS z"; + + @Override + public String marshal(Date date) throws Exception { + final SimpleDateFormat formatter = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT); + formatter.setTimeZone(TimeZone.getDefault()); + return formatter.format(date); + } + + @Override + public Date unmarshal(String date) throws Exception { + final SimpleDateFormat parser = new SimpleDateFormat(DEFAULT_DATE_TIME_FORMAT); + parser.setTimeZone(TimeZone.getDefault()); + return parser.parse(date); + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java new file mode 100644 index 0000000000..36fc163a04 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AboutEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.AboutDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a AboutDTO. + */ +@XmlRootElement(name = "aboutEntity") +public class AboutEntity extends Entity { + + private AboutDTO about; + + /** + * The AboutDTO that is being serialized. + * + * @return The AboutDTO object + */ + public AboutDTO getAbout() { + return about; + } + + public void setAbout(AboutDTO about) { + this.about = about; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java new file mode 100644 index 0000000000..1b2c0b63c8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ActionEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.action.ActionDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to an ActionDTO. + */ +@XmlRootElement(name = "actionEntity") +public class ActionEntity extends Entity { + + private ActionDTO action; + + /** + * The ActionDTO that is being serialized. + * + * @return The ActionDTO object + */ + public ActionDTO getAction() { + return action; + } + + public void setAction(ActionDTO action) { + this.action = action; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java new file mode 100644 index 0000000000..eed1305526 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/AuthorityEntity.java @@ -0,0 +1,60 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a set of user + * authorities. + */ +@XmlRootElement(name = "authoritiesEntity") +public class AuthorityEntity extends Entity { + + private String userId; + private Set authorities; + + /** + * The current user id. + * + * @return + */ + public String getUserId() { + return userId; + } + + public void setUserId(String userId) { + this.userId = userId; + } + + /** + * The set of authorities that are being serialized. + * + * @return + */ + public Set getAuthorities() { + return authorities; + } + + public void setAuthorities(Set authorities) { + this.authorities = authorities; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.java new file mode 100644 index 0000000000..594707a6df --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BannerEntity.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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.nifi.web.api.dto.BannerDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a BannerDTO. + */ +@XmlRootElement(name = "bannersEntity") +public class BannerEntity extends Entity { + + private BannerDTO banners; + + /** + * The BannerDTO that is being serialized. + * + * @return The BannerDTO object + */ + public BannerDTO getBanners() { + return banners; + } + + public void setBanners(BannerDTO about) { + this.banners = about; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java new file mode 100644 index 0000000000..4287893ffd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/BulletinBoardEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.BulletinBoardDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a BulletinBoardDTO. + */ +@XmlRootElement(name = "bulletinBoardEntity") +public class BulletinBoardEntity extends Entity { + + private BulletinBoardDTO messages; + + /** + * The BulletinBoardDTO that is being serialized. + * + * @return The BulletinBoardDTO object + */ + public BulletinBoardDTO getBulletinBoard() { + return messages; + } + + public void setBulletinBoard(BulletinBoardDTO messages) { + this.messages = messages; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java new file mode 100644 index 0000000000..732a4b7077 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterConnectionStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ClusterConnectionStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterConnectionStatusDTO. + */ +@XmlRootElement(name = "clusterConnectionStatusEntity") +public class ClusterConnectionStatusEntity extends Entity { + + private ClusterConnectionStatusDTO clusterConnectionStatus; + + /** + * The ClusterConnectionStatusDTO that is being serialized. + * + * @return The ClusterConnectionStatusDTO object + */ + public ClusterConnectionStatusDTO getClusterConnectionStatus() { + return clusterConnectionStatus; + } + + public void setClusterConnectionStatus(ClusterConnectionStatusDTO clusterConnectionStatus) { + this.clusterConnectionStatus = clusterConnectionStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java new file mode 100644 index 0000000000..8306b0f879 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ClusterDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterDTO. + */ +@XmlRootElement(name = "clusterEntity") +public class ClusterEntity extends Entity { + + private ClusterDTO cluster; + + /** + * The ClusterDTO that is being serialized. + * + * @return The ClusterDTO object + */ + public ClusterDTO getCluster() { + return cluster; + } + + public void setCluster(ClusterDTO cluster) { + this.cluster = cluster; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java new file mode 100644 index 0000000000..16a249732b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterPortStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ClusterPortStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterPortStatusDTO. + */ +@XmlRootElement(name = "clusterPortStatusEntity") +public class ClusterPortStatusEntity extends Entity { + + private ClusterPortStatusDTO clusterPortStatus; + + /** + * The ClusterPortStatusDTO that is being serialized. + * + * @return The ClusterPortStatusDTO object + */ + public ClusterPortStatusDTO getClusterPortStatus() { + return clusterPortStatus; + } + + public void setClusterPortStatus(ClusterPortStatusDTO clusterPortStatus) { + this.clusterPortStatus = clusterPortStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java new file mode 100644 index 0000000000..68e5c5cac5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterProcessorStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ClusterProcessorStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterProcessorStatusDTO. + */ +@XmlRootElement(name = "clusterProcessorStatusEntity") +public class ClusterProcessorStatusEntity extends Entity { + + private ClusterProcessorStatusDTO clusterProcessorStatus; + + /** + * The ClusterProcessorStatusDTO that is being serialized. + * + * @return The ClusterProcessorStatusDTO object + */ + public ClusterProcessorStatusDTO getClusterProcessorStatus() { + return clusterProcessorStatus; + } + + public void setClusterProcessorStatus(ClusterProcessorStatusDTO clusterProcessorStatus) { + this.clusterProcessorStatus = clusterProcessorStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java new file mode 100644 index 0000000000..8a3da7cbc3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterRemoteProcessGroupStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ClusterRemoteProcessGroupStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterRemoteProcessGroupStatusDTO. + */ +@XmlRootElement(name = "clusterRemoteProcessGroupStatusEntity") +public class ClusterRemoteProcessGroupStatusEntity extends Entity { + + private ClusterRemoteProcessGroupStatusDTO clusterRemoteProcessGroupStatus; + + /** + * The ClusterRemoteProcessGroupStatusDTO that is being serialized. + * + * @return The ClusterRemoteProcessGroupStatusDTO object + */ + public ClusterRemoteProcessGroupStatusDTO getClusterRemoteProcessGroupStatus() { + return clusterRemoteProcessGroupStatus; + } + + public void setClusterRemoteProcessGroupStatus(ClusterRemoteProcessGroupStatusDTO clusterRemoteProcessGroupStatus) { + this.clusterRemoteProcessGroupStatus = clusterRemoteProcessGroupStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.java new file mode 100644 index 0000000000..546bd8b9f5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterSearchResultsEntity.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.web.api.entity; + +import java.util.List; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.search.NodeSearchResultDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to NodeSearchResultDTOs. + */ +@XmlRootElement(name = "clusterSearchResultsEntity") +public class ClusterSearchResultsEntity { + + private List nodeResults; + + /** + * The node search results. + * + * @return + */ + public List getNodeResults() { + return nodeResults; + } + + public void setNodeResults(List nodeResults) { + this.nodeResults = nodeResults; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java new file mode 100644 index 0000000000..d22d432515 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ClusterStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterStatusDTO. + */ +@XmlRootElement(name = "clusterStatusEntity") +public class ClusterStatusEntity extends Entity { + + private ClusterStatusDTO clusterStatus; + + /** + * The ClusterStatusDTO that is being serialized. + * + * @return The ClusterStatusDTO object + */ + public ClusterStatusDTO getClusterStatus() { + return clusterStatus; + } + + public void setClusterStatus(ClusterStatusDTO clusterStatus) { + this.clusterStatus = clusterStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java new file mode 100644 index 0000000000..9ed69fd807 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ClusterStatusHistoryEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ClusterStatusHistoryDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ClusterStatusHistoryDTO. + */ +@XmlRootElement(name = "clusterStatusHistoryEntity") +public class ClusterStatusHistoryEntity extends Entity { + + private ClusterStatusHistoryDTO clusterStatusHistory; + + /** + * The ClusterStatusHistoryDTO that is being serialized. + * + * @return The ClusterStatusHistoryDTO object + */ + public ClusterStatusHistoryDTO getClusterStatusHistory() { + return clusterStatusHistory; + } + + public void setClusterStatusHistory(ClusterStatusHistoryDTO clusterStatusHistory) { + this.clusterStatusHistory = clusterStatusHistory; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java new file mode 100644 index 0000000000..69ae937e21 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ConnectionDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a + * ConnectionDTO. + */ +@XmlRootElement(name = "connectionEntity") +public class ConnectionEntity extends Entity { + + private ConnectionDTO connection; + + /** + * The RelationshipDTO that is being serialized. + * + * @return + */ + public ConnectionDTO getConnection() { + return connection; + } + + public void setConnection(ConnectionDTO connection) { + this.connection = connection; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java new file mode 100644 index 0000000000..c69c69ae3d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ConnectionsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ConnectionDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * ConnectionDTOs. + */ +@XmlRootElement(name = "connectionsEntity") +public class ConnectionsEntity extends Entity { + + private Set connections; + + /** + * The list of ConnectionDTOs that are being serialized. + * + * @return + */ + public Set getConnections() { + return connections; + } + + public void setConnections(Set connections) { + this.connections = connections; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java new file mode 100644 index 0000000000..e13903b267 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerConfigurationEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerConfigurationDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ControllerConfigurationDTO. + */ +@XmlRootElement(name = "controllerConfigurationEntity") +public class ControllerConfigurationEntity extends Entity { + + private ControllerConfigurationDTO config; + + /** + * The ControllerConfigurationDTO that is being serialized. + * + * @return The ControllerConfigurationDTO object + */ + public ControllerConfigurationDTO getConfig() { + return config; + } + + public void setConfig(ControllerConfigurationDTO config) { + this.config = config; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java new file mode 100644 index 0000000000..62d0409efe --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ControllerDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ControllerDTO. + */ +@XmlRootElement(name = "controllerEntity") +public class ControllerEntity extends Entity { + + private ControllerDTO controller; + + /** + * The ControllerDTO that is being serialized. + * + * @return The ControllerDTO object + */ + public ControllerDTO getController() { + return controller; + } + + public void setController(ControllerDTO controller) { + this.controller = controller; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java new file mode 100644 index 0000000000..94d8aca23f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ControllerStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ControllerStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ControllerStatusDTO. + */ +@XmlRootElement(name = "controllerStatusEntity") +public class ControllerStatusEntity extends Entity { + + private ControllerStatusDTO controllerStatus; + + /** + * The ControllerStatusDTO that is being serialized. + * + * @return The ControllerStatusDTO object + */ + public ControllerStatusDTO getControllerStatus() { + return controllerStatus; + } + + public void setControllerStatus(ControllerStatusDTO controllerStatus) { + this.controllerStatus = controllerStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java new file mode 100644 index 0000000000..666dc75fea --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CounterEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.CounterDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response from the API. This particular entity holds a reference to a + * CounterDTO. + */ +@XmlRootElement(name = "counterEntity") +public class CounterEntity extends Entity { + + private CounterDTO counter; + + /** + * Get the counter. + * + * @return + */ + public CounterDTO getCounter() { + return counter; + } + + public void setCounter(CounterDTO counter) { + this.counter = counter; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.java new file mode 100644 index 0000000000..bee76692fc --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/CountersEntity.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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.CountersDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response from the API. This particular entity holds a reference to a + * CountersDTO. + */ +@XmlRootElement(name = "countersEntity") +public class CountersEntity extends Entity { + + private CountersDTO counters; + + /** + * Get the counters which contains all the counter groups and a generation + * date. + * + * @return + */ + public CountersDTO getCounters() { + return counters; + } + + public void setCounters(CountersDTO counters) { + this.counters = counters; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java new file mode 100644 index 0000000000..6abcbf391a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/Entity.java @@ -0,0 +1,43 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.RevisionDTO; + +/** + * A base type for request/response entities. + */ +@XmlRootElement(name = "entity") +public class Entity { + + private RevisionDTO revision; + + /** + * A revision for this request/response. + * + * @return + */ + public RevisionDTO getRevision() { + return revision; + } + + public void setRevision(RevisionDTO revision) { + this.revision = revision; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java new file mode 100644 index 0000000000..a15cc51de3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FlowSnippetEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.FlowSnippetDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a FlowSnippetDTO. + */ +@XmlRootElement(name = "flowSnippetEntity") +public class FlowSnippetEntity extends Entity { + + private FlowSnippetDTO contents; + + /** + * The SnippetDTO that is being serialized. + * + * @return The SnippetDTO object + */ + public FlowSnippetDTO getContents() { + return contents; + } + + public void setContents(FlowSnippetDTO contents) { + this.contents = contents; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java new file mode 100644 index 0000000000..8b43fb8bbd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.FunnelDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a FunnelDTO. + */ +@XmlRootElement(name = "funnelEntity") +public class FunnelEntity extends Entity { + + private FunnelDTO funnel; + + /** + * The FunnelDTO that is being serialized. + * + * @return The FunnelDTO object + */ + public FunnelDTO getFunnel() { + return funnel; + } + + public void setFunnel(FunnelDTO funnel) { + this.funnel = funnel; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java new file mode 100644 index 0000000000..2b8cbc88db --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/FunnelsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.FunnelDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * FunnelDTOs. + */ +@XmlRootElement(name = "funnelsEntity") +public class FunnelsEntity extends Entity { + + private Set funnels; + + /** + * The collection of FunnelDTOs that are being serialized. + * + * @return + */ + public Set getFunnels() { + return funnels; + } + + public void setFunnels(Set labels) { + this.funnels = labels; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java new file mode 100644 index 0000000000..f817cce4bd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/HistoryEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.action.HistoryDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a HistoryDTO. + */ +@XmlRootElement(name = "historyEntity") +public class HistoryEntity extends Entity { + + private HistoryDTO history; + + /** + * The HistoryDTO that is being serialized. + * + * @return The LabelDTO object + */ + public HistoryDTO getHistory() { + return history; + } + + public void setHistory(HistoryDTO history) { + this.history = history; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java new file mode 100644 index 0000000000..9fec9d8727 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.PortDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to an input + * PortDTO. + */ +@XmlRootElement(name = "inputPortEntity") +public class InputPortEntity extends Entity { + + private PortDTO inputPort; + + /** + * The input PortDTO that are being serialized. + * + * @return + */ + public PortDTO getInputPort() { + return inputPort; + } + + public void setInputPort(PortDTO inputPort) { + this.inputPort = inputPort; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java new file mode 100644 index 0000000000..8637596284 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/InputPortsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.PortDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * input PortDTOs. + */ +@XmlRootElement(name = "inputPortsEntity") +public class InputPortsEntity extends Entity { + + private Set inputPorts; + + /** + * The collection of input PortDTOs that are being serialized. + * + * @return + */ + public Set getInputPorts() { + return inputPorts; + } + + public void setInputPorts(Set inputPorts) { + this.inputPorts = inputPorts; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java new file mode 100644 index 0000000000..867d781402 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.LabelDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a LabelDTO. + */ +@XmlRootElement(name = "labelEntity") +public class LabelEntity extends Entity { + + private LabelDTO label; + + /** + * The LabelDTO that is being serialized. + * + * @return The LabelDTO object + */ + public LabelDTO getLabel() { + return label; + } + + public void setLabel(LabelDTO label) { + this.label = label; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java new file mode 100644 index 0000000000..063eeafe40 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LabelsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.LabelDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * LabelDTOs. + */ +@XmlRootElement(name = "labelsEntity") +public class LabelsEntity extends Entity { + + private Set labels; + + /** + * The collection of LabelDTOs that are being serialized. + * + * @return + */ + public Set getLabels() { + return labels; + } + + public void setLabels(Set labels) { + this.labels = labels; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java new file mode 100644 index 0000000000..5d443af89b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/LineageEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.provenance.lineage.LineageDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a LineageDTO. + */ +@XmlRootElement(name = "lineageEntity") +public class LineageEntity extends Entity { + + private LineageDTO lineage; + + /** + * The LineageDTO that is being serialized. + * + * @return The LineageDTO object + */ + public LineageDTO getLineage() { + return lineage; + } + + public void setLineage(LineageDTO lineage) { + this.lineage = lineage; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java new file mode 100644 index 0000000000..ac5b306f16 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.NodeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a NodeDTO. + */ +@XmlRootElement(name = "nodeEntity") +public class NodeEntity extends Entity { + + private NodeDTO node; + + /** + * The NodeDTO that is being serialized. + * + * @return The NodeDTO object + */ + public NodeDTO getNode() { + return node; + } + + public void setNode(NodeDTO node) { + this.node = node; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java new file mode 100644 index 0000000000..d7471dae67 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.NodeStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a NodeStatusDTO. + */ +@XmlRootElement(name = "nodeStatusEntity") +public class NodeStatusEntity extends Entity { + + private NodeStatusDTO nodeStatus; + + /** + * The NodeStatusDTO that is being serialized. + * + * @return The NodeStatusDTO object + */ + public NodeStatusDTO getNodeStatus() { + return nodeStatus; + } + + public void setNodeStatus(NodeStatusDTO nodeStatus) { + this.nodeStatus = nodeStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java new file mode 100644 index 0000000000..59eafa16b7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/NodeSystemDiagnosticsEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.NodeSystemDiagnosticsDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a NodeSystemDiagnosticsDTO. + */ +@XmlRootElement(name = "nodeSystemDiagnosticsEntity") +public class NodeSystemDiagnosticsEntity extends Entity { + + private NodeSystemDiagnosticsDTO nodeSystemDiagnostics; + + /** + * The NodeSystemDiagnosticsDTO that is being serialized. + * + * @return The NodeSystemDiagnosticsDTO object + */ + public NodeSystemDiagnosticsDTO getNodeSystemDiagnostics() { + return nodeSystemDiagnostics; + } + + public void setNodeSystemDiagnostics(NodeSystemDiagnosticsDTO nodeSystemDiagnostics) { + this.nodeSystemDiagnostics = nodeSystemDiagnostics; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java new file mode 100644 index 0000000000..07fdab7589 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.PortDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to an output + * PortDTO. + */ +@XmlRootElement(name = "outputPortEntity") +public class OutputPortEntity extends Entity { + + private PortDTO outputPort; + + /** + * The output PortDTO that are being serialized. + * + * @return + */ + public PortDTO getOutputPort() { + return outputPort; + } + + public void setOutputPort(PortDTO outputPort) { + this.outputPort = outputPort; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java new file mode 100644 index 0000000000..9fa398dadd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/OutputPortsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.PortDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * output PortDTOs. + */ +@XmlRootElement(name = "outputPortsEntity") +public class OutputPortsEntity extends Entity { + + private Set outputPorts; + + /** + * The collection of output PortDTOs that are being serialized. + * + * @return + */ + public Set getOutputPorts() { + return outputPorts; + } + + public void setOutputPorts(Set outputPorts) { + this.outputPorts = outputPorts; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.java new file mode 100644 index 0000000000..2ddddd8887 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/PrioritizerTypesEntity.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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * prioritizer types. + */ +@XmlRootElement(name = "prioritizerTypesEntity") +public class PrioritizerTypesEntity extends Entity { + + private Set prioritizerTypes; + + /** + * The list of prioritizer types that are being serialized. + * + * @return + */ + public Set getPrioritizerTypes() { + return prioritizerTypes; + } + + public void setPrioritizerTypes(Set prioritizerTypes) { + this.prioritizerTypes = prioritizerTypes; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java new file mode 100644 index 0000000000..c677ef1a24 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessGroupDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ProcessGroupDTO. + */ +@XmlRootElement(name = "processGroupEntity") +public class ProcessGroupEntity extends Entity { + + private ProcessGroupDTO processGroup; + + /** + * The ProcessGroupDTO that is being serialized. + * + * @return The ControllerDTO object + */ + public ProcessGroupDTO getProcessGroup() { + return processGroup; + } + + public void setProcessGroup(ProcessGroupDTO controller) { + this.processGroup = controller; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java new file mode 100644 index 0000000000..f0a6e0f651 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupStatusEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.ProcessGroupStatusDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ProcessGroupStatusDTO. + */ +@XmlRootElement(name = "processGroupStatusEntity") +public class ProcessGroupStatusEntity extends Entity { + + private ProcessGroupStatusDTO processGroupStatus; + + /** + * The ProcessGroupStatusDTO that is being serialized. + * + * @return The ProcessGroupStatusDTO object + */ + public ProcessGroupStatusDTO getProcessGroupStatus() { + return processGroupStatus; + } + + public void setProcessGroupStatus(ProcessGroupStatusDTO processGroupStatus) { + this.processGroupStatus = processGroupStatus; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.java new file mode 100644 index 0000000000..1fdf23f839 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessGroupsEntity.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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessGroupDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a set of ProcessGroupDTOs. + */ +@XmlRootElement(name = "processGroupsEntity") +public class ProcessGroupsEntity extends Entity { + + private Set processGroups; + + /** + * The ProcessGroupDTO that is being serialized. + * + * @return The ProcessGroupDTOs + */ + public Set getProcessGroups() { + return processGroups; + } + + public void setProcessGroups(Set processGroups) { + this.processGroups = processGroups; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java new file mode 100644 index 0000000000..21ad431df4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessorDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ProcessorDTO. + */ +@XmlRootElement(name = "processorEntity") +public class ProcessorEntity extends Entity { + + private ProcessorDTO processor; + + /** + * The ProcessorDTO that is being serialized. + * + * @return The ProcessorDTO object + */ + public ProcessorDTO getProcessor() { + return processor; + } + + public void setProcessor(ProcessorDTO processor) { + this.processor = processor; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java new file mode 100644 index 0000000000..19166f7c99 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorHistoryEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessorHistoryDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ProcessorHistoryDTO. + */ +@XmlRootElement(name = "processorHistoryEntity") +public class ProcessorHistoryEntity extends Entity { + + private ProcessorHistoryDTO propertyHistory; + + /** + * The ProcessorHistoryDTO that is being serialized. + * + * @return The ProcessorHistoryDTO object + */ + public ProcessorHistoryDTO getProcessorHistory() { + return propertyHistory; + } + + public void setProcessorHistory(ProcessorHistoryDTO propertyHistory) { + this.propertyHistory = propertyHistory; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.java new file mode 100644 index 0000000000..23237fe7e1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorTypesEntity.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.web.api.entity; + +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.DocumentedTypeDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * processor types. + */ +@XmlRootElement(name = "processorTypesEntity") +public class ProcessorTypesEntity extends Entity { + + private Set processorTypes; + + /** + * The list of processor types that are being serialized. + * + * @return + */ + public Set getProcessorTypes() { + return processorTypes; + } + + public void setProcessorTypes(Set processorTypes) { + this.processorTypes = processorTypes; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java new file mode 100644 index 0000000000..4962b38a8c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProcessorsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.ProcessorDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * ProcessorDTOs. + */ +@XmlRootElement(name = "processorsEntity") +public class ProcessorsEntity extends Entity { + + private Set processors; + + /** + * The collection of ProcessorDTOs that are being serialized. + * + * @return + */ + public Set getProcessors() { + return processors; + } + + public void setProcessors(Set processors) { + this.processors = processors; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java new file mode 100644 index 0000000000..72a85285fb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEntity.java @@ -0,0 +1,40 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO; + +@XmlRootElement(name = "provenanceEntity") +public class ProvenanceEntity extends Entity { + + private ProvenanceDTO provenance; + + /** + * The provenance . + * + * @return + */ + public ProvenanceDTO getProvenance() { + return provenance; + } + + public void setProvenance(ProvenanceDTO provenance) { + this.provenance = provenance; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java new file mode 100644 index 0000000000..0aa0a5528d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceEventEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ProvenanceEventDTO. + */ +@XmlRootElement(name = "provenanceEventEntity") +public class ProvenanceEventEntity extends Entity { + + private ProvenanceEventDTO provenanceEvent; + + /** + * The ProvenanceEventDTO that is being serialized. + * + * @return The ProvenanceEventDTO object + */ + public ProvenanceEventDTO getProvenanceEvent() { + return provenanceEvent; + } + + public void setProvenanceEvent(ProvenanceEventDTO provenanceEvent) { + this.provenanceEvent = provenanceEvent; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.java new file mode 100644 index 0000000000..01591cb3a8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/ProvenanceOptionsEntity.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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.nifi.web.api.dto.provenance.ProvenanceOptionsDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a ProvenanceOptionsDTO. + */ +@XmlRootElement(name = "provenanceOptionsEntity") +public class ProvenanceOptionsEntity extends Entity { + + private ProvenanceOptionsDTO provenanceOptions; + + /** + * The ProvenanceOptionsDTO that is being serialized. + * + * @return The ProvenanceOptionsDTO object + */ + public ProvenanceOptionsDTO getProvenanceOptions() { + return provenanceOptions; + } + + public void setProvenanceOptions(ProvenanceOptionsDTO provenanceOptions) { + this.provenanceOptions = provenanceOptions; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java new file mode 100644 index 0000000000..62a84d6abb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a RemoteProcessGroupDTO. + */ +@XmlRootElement(name = "remoteProcessGroupEntity") +public class RemoteProcessGroupEntity extends Entity { + + private RemoteProcessGroupDTO remoteProcessGroup; + + /** + * The RemoteProcessGroupDTO that is being serialized. + * + * @return The RemoteProcessGroupDTO object + */ + public RemoteProcessGroupDTO getRemoteProcessGroup() { + return remoteProcessGroup; + } + + public void setRemoteProcessGroup(RemoteProcessGroupDTO remoteProcessGroup) { + this.remoteProcessGroup = remoteProcessGroup; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java new file mode 100644 index 0000000000..f310b5e5ec --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupPortEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a RemoteProcessGroupPortDTO. + */ +@XmlRootElement(name = "remoteProcessGroupPortEntity") +public class RemoteProcessGroupPortEntity extends Entity { + + private RemoteProcessGroupPortDTO remoteProcessGroupPort; + + /** + * The RemoteProcessGroupPortDTO that is being serialized. + * + * @return The RemoteProcessGroupPortDTO object + */ + public RemoteProcessGroupPortDTO getRemoteProcessGroupPort() { + return remoteProcessGroupPort; + } + + public void setRemoteProcessGroupPort(RemoteProcessGroupPortDTO remoteProcessGroupPort) { + this.remoteProcessGroupPort = remoteProcessGroupPort; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java new file mode 100644 index 0000000000..a04c789636 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/RemoteProcessGroupsEntity.java @@ -0,0 +1,47 @@ +/* + * 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.web.api.entity; + +import java.util.Set; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a response to the API. This particular entity holds a reference to a list of + * RemoteProcessGroupDTOs. + */ +@XmlRootElement(name = "remoteProcessGroupsEntity") +public class RemoteProcessGroupsEntity extends Entity { + + private Set remoteProcessGroups; + + /** + * The collection of RemoteProcessGroupDTOs that are being serialized. + * + * @return + */ + public Set getRemoteProcessGroups() { + return remoteProcessGroups; + } + + public void setRemoteProcessGroups(Set remoteProcessGroups) { + this.remoteProcessGroups = remoteProcessGroups; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.java new file mode 100644 index 0000000000..6b9a88b3ff --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SearchResultsEntity.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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.search.SearchResultsDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to ProcessorSearchResultDTOs, RemoteProcessGroupSearchResultDTOs, + * and ConnectionSearchResultDTOs. + */ +@XmlRootElement(name = "searchResultsEntity") +public class SearchResultsEntity { + + private SearchResultsDTO searchResultsDTO; + + /** + * The search results. + * + * @return + */ + public SearchResultsDTO getSearchResultsDTO() { + return searchResultsDTO; + } + + public void setSearchResultsDTO(SearchResultsDTO searchResultsDTO) { + this.searchResultsDTO = searchResultsDTO; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java new file mode 100644 index 0000000000..a9b05a1cc3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SnippetEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.SnippetDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a SnippetDTO. + */ +@XmlRootElement(name = "snippetEntity") +public class SnippetEntity extends Entity { + + private SnippetDTO snippet; + + /** + * The SnippetDTO that is being serialized. + * + * @return The SnippetDTO object + */ + public SnippetDTO getSnippet() { + return snippet; + } + + public void setSnippet(SnippetDTO snippet) { + this.snippet = snippet; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java new file mode 100644 index 0000000000..d3c2247d3a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/StatusHistoryEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a StatusHistoryDTO. + */ +@XmlRootElement(name = "statusHistoryEntity") +public class StatusHistoryEntity extends Entity { + + private StatusHistoryDTO statusHistory; + + /** + * The StatusHistoryDTO that is being serialized. + * + * @return The StatusHistoryDTO object + */ + public StatusHistoryDTO getStatusHistory() { + return statusHistory; + } + + public void setStatusHistory(StatusHistoryDTO statusHistory) { + this.statusHistory = statusHistory; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java new file mode 100644 index 0000000000..e7baa1606b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/SystemDiagnosticsEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.SystemDiagnosticsDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a SystemDiagnosticsDTO. + */ +@XmlRootElement(name = "systemDiagnosticsEntity") +public class SystemDiagnosticsEntity extends Entity { + + private SystemDiagnosticsDTO systemDiagnostics; + + /** + * The SystemDiagnosticsDTO that is being serialized. + * + * @return The SystemDiagnosticsDTO object + */ + public SystemDiagnosticsDTO getSystemDiagnostics() { + return systemDiagnostics; + } + + public void setSystemDiagnostics(SystemDiagnosticsDTO health) { + this.systemDiagnostics = health; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java new file mode 100644 index 0000000000..8e9f07a2a8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplateEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.TemplateDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a TemplateDTO. + */ +@XmlRootElement(name = "templateEntity") +public class TemplateEntity extends Entity { + + private TemplateDTO template; + + /** + * The TemplateDTO that is being serialized. + * + * @return The TemplateDTO object + */ + public TemplateDTO getTemplate() { + return template; + } + + public void setTemplate(TemplateDTO template) { + this.template = template; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java new file mode 100644 index 0000000000..34000453f2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/TemplatesEntity.java @@ -0,0 +1,63 @@ +/* + * 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.web.api.entity; + +import java.util.Date; +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.TemplateDTO; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a set of TemplateDTOs. + */ +@XmlRootElement(name = "templatesEntity") +public class TemplatesEntity extends Entity { + + private Set templates; + private Date generated; + + /** + * The set of TemplateDTOs that is being serialized. + * + * @return The TemplateDTO object + */ + public Set getTemplates() { + return templates; + } + + public void setTemplates(Set templates) { + this.templates = templates; + } + + /** + * When this content was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java new file mode 100644 index 0000000000..963e853e50 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.UserDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a UserDTO. + */ +@XmlRootElement(name = "userEntity") +public class UserEntity extends Entity { + + private UserDTO user; + + /** + * The UserDTO that is being serialized. + * + * @return The UserDTO object + */ + public UserDTO getUser() { + return user; + } + + public void setUser(UserDTO user) { + this.user = user; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java new file mode 100644 index 0000000000..a6542c8f8b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserGroupEntity.java @@ -0,0 +1,45 @@ +/* + * 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.web.api.entity; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.UserGroupDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a UserGroupDTO. + */ +@XmlRootElement(name = "userGroupEntity") +public class UserGroupEntity extends Entity { + + private UserGroupDTO userGroup; + + /** + * The UserGroupDTO that is being serialized. + * + * @return The UserGroupDTO object + */ + public UserGroupDTO getUserGroup() { + return userGroup; + } + + public void setUserGroup(UserGroupDTO userGroup) { + this.userGroup = userGroup; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java new file mode 100644 index 0000000000..baffe1511c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UserSearchResultsEntity.java @@ -0,0 +1,61 @@ +/* + * 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.web.api.entity; + +import java.util.List; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.web.api.dto.search.UserGroupSearchResultDTO; +import org.apache.nifi.web.api.dto.search.UserSearchResultDTO; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to UserSearchResultDTOs and UserGroupSearchResultDTOs. + */ +@XmlRootElement(name = "userSearchResultsEntity") +public class UserSearchResultsEntity { + + private List userResults; + private List userGroupResults; + + /** + * The user search results. + * + * @return + */ + public List getUserResults() { + return userResults; + } + + public void setUserResults(List userResults) { + this.userResults = userResults; + } + + /** + * The user group search results. + * + * @return + */ + public List getUserGroupResults() { + return userGroupResults; + } + + public void setUserGroupResults(List userGroupResults) { + this.userGroupResults = userGroupResults; + } + +} diff --git a/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java new file mode 100644 index 0000000000..2d11d1f0cb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/client-dto/src/main/java/org/apache/nifi/web/api/entity/UsersEntity.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.web.api.entity; + +import java.util.Collection; +import java.util.Date; +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.web.api.dto.UserDTO; +import org.apache.nifi.web.api.dto.util.TimeAdapter; + +/** + * A serialized representation of this class can be placed in the entity body of + * a request or response to or from the API. This particular entity holds a + * reference to a collection of UserDTO. + */ +@XmlRootElement(name = "usersEntity") +public class UsersEntity extends Entity { + + private Collection users; + private Date generated; + + /** + * The collection of UserDTOs that are being serialized. + * + * @return The UserDTO object + */ + public Collection getUsers() { + return users; + } + + public void setUsers(Collection users) { + this.users = users; + } + + /** + * When this content was generated. + * + * @return + */ + @XmlJavaTypeAdapter(TimeAdapter.class) + public Date getGenerated() { + return generated; + } + + public void setGenerated(Date generated) { + this.generated = generated; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml new file mode 100644 index 0000000000..b8960c34de --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/pom.xml @@ -0,0 +1,48 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + + cluster-authorization-provider + NiFi Framework Cluster Authority Provider + + + org.apache.nifi + nifi-api + + + org.apache.nifi + file-authorization-provider + + + org.apache.nifi + framework-cluster-protocol + + + org.apache.nifi + framework-cluster + + + org.apache.nifi + nifi-socket-utils + + + diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java new file mode 100644 index 0000000000..2b3b38c4c3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/ClusterManagerAuthorizationProvider.java @@ -0,0 +1,225 @@ +/* + * 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.cluster.authorization; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.AuthorityProviderConfigurationContext; +import org.apache.nifi.authorization.AuthorityProviderInitializationContext; +import org.apache.nifi.authorization.FileAuthorizationProvider; +import org.apache.nifi.authorization.annotation.AuthorityProviderContext; +import org.apache.nifi.authorization.exception.ProviderCreationException; +import org.apache.nifi.authorization.exception.ProviderDestructionException; +import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage; +import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage; +import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage; +import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage; +import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.DOES_DN_EXIST; +import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.GET_AUTHORITIES; +import static org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType.GET_GROUP_FOR_USER; +import org.apache.nifi.cluster.authorization.protocol.message.jaxb.JaxbProtocolUtils; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketListener; +import org.apache.nifi.io.socket.SocketUtils; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.util.NiFiProperties; +import static org.apache.nifi.util.NiFiProperties.CLUSTER_MANAGER_ADDRESS; +import org.apache.nifi.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.BeansException; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * Provides authorities for the NCM in clustered environments. Communication + * occurs over TCP/IP sockets. All method calls are deferred to the + * FileAuthorizationProvider. + */ +public class ClusterManagerAuthorizationProvider extends FileAuthorizationProvider implements AuthorityProvider, ApplicationContextAware { + + public static final String AUTHORITY_PROVIDER_SERVIVE_NAME = "cluster-authority-provider"; + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterManagerAuthorizationProvider.class)); + private static final String CLUSTER_MANAGER_AUTHORITY_PROVIDER_PORT = "Authority Provider Port"; + private static final String CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS = "Authority Provider Threads"; + private static final int DEFAULT_CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS = 10; + + private WebClusterManager clusterManager; + private ProtocolContext authorityProviderProtocolContext; + private SocketListener socketListener; + private NiFiProperties properties; + private ApplicationContext applicationContext; + + @Override + public void initialize(final AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException { + super.initialize(initializationContext); + } + + @Override + public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException { + super.onConfigured(configurationContext); + + // get the socket address of the cluster authority provider + final InetSocketAddress clusterAuthorityProviderAddress = getClusterManagerAuthorityProviderAddress(configurationContext); + + // get the cluster manager + clusterManager = applicationContext.getBean("clusterManager", WebClusterManager.class); + + // if using multicast, then the authority provider's service is broadcasted + if (properties.getClusterProtocolUseMulticast()) { + + // create the authority provider service for discovery + final DiscoverableService clusterAuthorityProviderService = new DiscoverableServiceImpl(AUTHORITY_PROVIDER_SERVIVE_NAME, clusterAuthorityProviderAddress); + + // register the authority provider service with the cluster manager + clusterManager.addBroadcastedService(clusterAuthorityProviderService); + } + + // get the number of protocol listening thread + final int numThreads = getClusterManagerAuthorityProviderThreads(configurationContext); + + // the server socket configuration + final ServerSocketConfiguration configuration = applicationContext.getBean("protocolServerSocketConfiguration", ServerSocketConfiguration.class); + + // the authority provider listens for node messages + socketListener = new SocketListener(numThreads, clusterAuthorityProviderAddress.getPort(), configuration) { + @Override + public void dispatchRequest(final Socket socket) { + ClusterManagerAuthorizationProvider.this.dispatchRequest(socket); + } + }; + + // start the socket listener + if (socketListener != null && !socketListener.isRunning()) { + try { + socketListener.start(); + } catch (final IOException ioe) { + throw new ProviderCreationException("Failed to start Cluster Manager Authorization Provider due to: " + ioe, ioe); + } + } + + // initialize the protocol context + authorityProviderProtocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + } + + @Override + public void preDestruction() throws ProviderDestructionException { + if (socketListener != null && socketListener.isRunning()) { + try { + socketListener.stop(); + } catch (final IOException ioe) { + throw new ProviderDestructionException("Failed to stop Cluster Manager Authorization Provider due to: " + ioe, ioe); + } + } + super.preDestruction(); + } + + private int getClusterManagerAuthorityProviderThreads(final AuthorityProviderConfigurationContext configurationContext) { + try { + return Integer.parseInt(configurationContext.getProperty(CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS)); + } catch (NumberFormatException nfe) { + return DEFAULT_CLUSTER_MANAGER_AUTHORITY_PROVIDER_THREADS; + } + } + + private InetSocketAddress getClusterManagerAuthorityProviderAddress(final AuthorityProviderConfigurationContext configurationContext) { + try { + String socketAddress = properties.getProperty(CLUSTER_MANAGER_ADDRESS); + if (StringUtils.isBlank(socketAddress)) { + socketAddress = "localhost"; + } + return InetSocketAddress.createUnresolved(socketAddress, getClusterManagerAuthorityProviderPort(configurationContext)); + } catch (Exception ex) { + throw new RuntimeException("Invalid manager authority provider address/port due to: " + ex, ex); + } + } + + private Integer getClusterManagerAuthorityProviderPort(final AuthorityProviderConfigurationContext configurationContext) { + final String authorityProviderPort = configurationContext.getProperty(CLUSTER_MANAGER_AUTHORITY_PROVIDER_PORT); + if (authorityProviderPort == null || authorityProviderPort.trim().isEmpty()) { + throw new ProviderCreationException("The authority provider port must be specified."); + } + + return Integer.parseInt(authorityProviderPort); + } + + private void dispatchRequest(final Socket socket) { + try { + // unmarshall message + final ProtocolMessageUnmarshaller unmarshaller = authorityProviderProtocolContext.createUnmarshaller(); + final ProtocolMessage request = unmarshaller.unmarshal(socket.getInputStream()); + final ProtocolMessage response = request; + + try { + switch (request.getType()) { + case DOES_DN_EXIST: { + final DoesDnExistMessage castedMsg = (DoesDnExistMessage) request; + castedMsg.setResponse(doesDnExist(castedMsg.getDn())); + break; + } + case GET_AUTHORITIES: { + final GetAuthoritiesMessage castedMsg = (GetAuthoritiesMessage) request; + castedMsg.setResponse(getAuthorities(castedMsg.getDn())); + break; + } + case GET_GROUP_FOR_USER: { + final GetGroupForUserMessage castedMsg = (GetGroupForUserMessage) request; + castedMsg.setResponse(getGroupForUser(castedMsg.getDn())); + break; + } + default: { + throw new Exception("Unsupported Message Type: " + request.getType()); + } + } + } catch (final Exception ex) { + response.setExceptionClass(ex.getClass().getName()); + response.setExceptionMessage(ex.getMessage()); + } + + final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller(); + marshaller.marshal(response, socket.getOutputStream()); + + } catch (final Exception e) { + logger.warn("Failed processing Socket Authorization Provider protocol message due to " + e, e); + } finally { + SocketUtils.closeQuietly(socket); + } + } + + @Override + @AuthorityProviderContext + public void setApplicationContext(final ApplicationContext applicationContext) throws BeansException { + this.applicationContext = applicationContext; + } + + @Override + @AuthorityProviderContext + public void setNiFiProperties(NiFiProperties properties) { + super.setNiFiProperties(properties); + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java new file mode 100644 index 0000000000..c81e9d0f4a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/NodeAuthorizationProvider.java @@ -0,0 +1,381 @@ +/* + * 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.cluster.authorization; + +import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage; +import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage; +import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import org.apache.nifi.authorization.Authority; +import org.apache.nifi.authorization.AuthorityProvider; +import org.apache.nifi.authorization.AuthorityProviderConfigurationContext; +import org.apache.nifi.authorization.AuthorityProviderInitializationContext; +import org.apache.nifi.authorization.annotation.AuthorityProviderContext; +import org.apache.nifi.authorization.exception.AuthorityAccessException; +import org.apache.nifi.authorization.exception.IdentityAlreadyExistsException; +import org.apache.nifi.authorization.exception.ProviderCreationException; +import org.apache.nifi.authorization.exception.ProviderDestructionException; +import org.apache.nifi.authorization.exception.UnknownIdentityException; +import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage; +import org.apache.nifi.cluster.authorization.protocol.message.jaxb.JaxbProtocolUtils; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.io.socket.SocketUtils; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery; +import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.util.NiFiProperties; +import static org.apache.nifi.util.NiFiProperties.CLUSTER_NODE_UNICAST_MANAGER_ADDRESS; +import org.apache.nifi.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.springframework.beans.BeansException; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * Provides authorities for nodes in clustered environments. Communication + * occurs over TCP/IP sockets. All method calls are communicated to the cluster + * manager provider via socket. + */ +public class NodeAuthorizationProvider implements AuthorityProvider, ApplicationContextAware { + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(NodeAuthorizationProvider.class)); + private static final String CLUSTER_NODE_MANAGER_AUTHORITY_PROVIDER_PORT = "Cluster Manager Authority Provider Port"; + + private ProtocolContext authorityProviderProtocolContext; + private SocketConfiguration socketConfiguration; + private ClusterServiceLocator serviceLocator; + private ApplicationContext applicationContext; + private NiFiProperties properties; + + @Override + public void initialize(AuthorityProviderInitializationContext initializationContext) throws ProviderCreationException { + } + + @Override + public void onConfigured(final AuthorityProviderConfigurationContext configurationContext) throws ProviderCreationException { + // TODO clear user cache? + + // if using multicast, then the authority provider's service is broadcasted + if (properties.getClusterProtocolUseMulticast()) { + // create the service discovery + final ClusterServiceDiscovery serviceDiscovery = new ClusterServiceDiscovery( + ClusterManagerAuthorizationProvider.AUTHORITY_PROVIDER_SERVIVE_NAME, + properties.getClusterProtocolMulticastAddress(), + applicationContext.getBean("protocolMulticastConfiguration", MulticastConfiguration.class), + applicationContext.getBean("protocolContext", ProtocolContext.class)); + + // create service location configuration + final ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); + config.setNumAttempts(3); + config.setTimeBetweenAttempts(1); + config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); + + serviceLocator = new ClusterServiceLocator(serviceDiscovery); + serviceLocator.setAttemptsConfig(config); + } else { + final InetSocketAddress serviceAddress = getClusterNodeManagerAuthorityProviderAddress(configurationContext); + final DiscoverableService service = new DiscoverableServiceImpl(ClusterManagerAuthorizationProvider.AUTHORITY_PROVIDER_SERVIVE_NAME, serviceAddress); + serviceLocator = new ClusterServiceLocator(service); + } + + try { + // start the service locator + serviceLocator.start(); + } catch (final IOException ioe) { + throw new ProviderCreationException(ioe); + } + + // the socket configuration + socketConfiguration = applicationContext.getBean("protocolSocketConfiguration", SocketConfiguration.class); + + // initialize the protocol context + authorityProviderProtocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + } + + private InetSocketAddress getClusterNodeManagerAuthorityProviderAddress(final AuthorityProviderConfigurationContext configurationContext) { + try { + String socketAddress = properties.getProperty(CLUSTER_NODE_UNICAST_MANAGER_ADDRESS); + if (StringUtils.isBlank(socketAddress)) { + socketAddress = "localhost"; + } + return InetSocketAddress.createUnresolved(socketAddress, getClusterNodeManagerAuthorityProviderPort(configurationContext)); + } catch (Exception ex) { + throw new ProviderCreationException("Invalid cluster manager authority provider address/port due to: " + ex, ex); + } + } + + private Integer getClusterNodeManagerAuthorityProviderPort(final AuthorityProviderConfigurationContext configurationContext) { + final String nodeAuthorityProviderPort = configurationContext.getProperty(CLUSTER_NODE_MANAGER_AUTHORITY_PROVIDER_PORT); + if (nodeAuthorityProviderPort == null || nodeAuthorityProviderPort.trim().isEmpty()) { + throw new ProviderCreationException("The cluster manager authority provider port must be specified."); + } + + return Integer.parseInt(nodeAuthorityProviderPort); + } + + @Override + public void setAuthorities(String dn, Set authorities) throws AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to set user authorities."); + } + + @Override + public void addUser(String dn, String group) throws IdentityAlreadyExistsException, AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to add users."); + } + + @Override + public boolean doesDnExist(String dn) throws AuthorityAccessException { + // create message + final DoesDnExistMessage msg = new DoesDnExistMessage(); + msg.setDn(dn); + + Socket socket = null; + try { + + final InetSocketAddress socketAddress = getServiceAddress(); + if (socketAddress == null) { + throw new AuthorityAccessException("Cluster Authority Provider's address is not known."); + } + + try { + // create a socket + socket = SocketUtils.createSocket(socketAddress, socketConfiguration); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe); + } + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + + try { + + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = authorityProviderProtocolContext.createUnmarshaller(); + final DoesDnExistMessage response = (DoesDnExistMessage) unmarshaller.unmarshal(socket.getInputStream()); + + // check if there was an exception + if (response.wasException()) { + throw new AuthorityAccessException(response.getExceptionMessage()); + } + + // return provider's response + return response.getResponse(); + + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe); + } + + } finally { + SocketUtils.closeQuietly(socket); + } + } + + @Override + public Set getAuthorities(String dn) throws UnknownIdentityException, AuthorityAccessException { + // create message + final GetAuthoritiesMessage msg = new GetAuthoritiesMessage(); + msg.setDn(dn); + + Socket socket = null; + try { + + final InetSocketAddress socketAddress = getServiceAddress(); + if (socketAddress == null) { + throw new AuthorityAccessException("Cluster Authority Provider's address is not known."); + } + + try { + // create a socket + socket = SocketUtils.createSocket(socketAddress, socketConfiguration); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe); + } + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + + try { + + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = authorityProviderProtocolContext.createUnmarshaller(); + final GetAuthoritiesMessage response = (GetAuthoritiesMessage) unmarshaller.unmarshal(socket.getInputStream()); + + // check if there was an exception + if (response.wasException()) { + if (isException(UnknownIdentityException.class, response)) { + throw new UnknownIdentityException(response.getExceptionMessage()); + } else { + throw new AuthorityAccessException(response.getExceptionMessage()); + } + } + + // return provider's response + return response.getResponse(); + + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe); + } + + } finally { + SocketUtils.closeQuietly(socket); + } + } + + @Override + public Set getUsers(Authority authority) throws AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to get users for a given authority."); + } + + @Override + public void revokeUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to revoke users."); + } + + @Override + public void setUsersGroup(Set dns, String group) throws UnknownIdentityException, AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to set user groups."); + } + + @Override + public void ungroupUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to ungroup users."); + } + + @Override + public void ungroup(String group) throws AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to ungroup."); + } + + @Override + public String getGroupForUser(String dn) throws UnknownIdentityException, AuthorityAccessException { + // create message + final GetGroupForUserMessage msg = new GetGroupForUserMessage(); + msg.setDn(dn); + + Socket socket = null; + try { + + final InetSocketAddress socketAddress = getServiceAddress(); + if (socketAddress == null) { + throw new AuthorityAccessException("Cluster Authority Provider's address is not known."); + } + + try { + // create a socket + socket = SocketUtils.createSocket(socketAddress, socketConfiguration); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed to create socket due to: " + ioe, ioe); + } + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = authorityProviderProtocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + + try { + + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = authorityProviderProtocolContext.createUnmarshaller(); + final GetGroupForUserMessage response = (GetGroupForUserMessage) unmarshaller.unmarshal(socket.getInputStream()); + + // check if there was an exception + if (response.wasException()) { + if (isException(UnknownIdentityException.class, response)) { + throw new UnknownIdentityException(response.getExceptionMessage()); + } else { + throw new AuthorityAccessException(response.getExceptionMessage()); + } + } + + return response.getResponse(); + } catch (final IOException ioe) { + throw new AuthorityAccessException("Failed unmarshalling '" + msg.getType() + "' response protocol message due to: " + ioe, ioe); + } + + } finally { + SocketUtils.closeQuietly(socket); + } + } + + @Override + public void revokeGroup(String group) throws UnknownIdentityException, AuthorityAccessException { + throw new AuthorityAccessException("Nodes are not allowed to revoke groups."); + } + + @Override + public void preDestruction() throws ProviderDestructionException { + try { + if (serviceLocator != null && serviceLocator.isRunning()) { + serviceLocator.stop(); + } + } catch (final IOException ioe) { + throw new ProviderDestructionException(ioe); + } + } + + @Override + @AuthorityProviderContext + public void setApplicationContext(final ApplicationContext applicationContext) throws BeansException { + this.applicationContext = applicationContext; + } + + @AuthorityProviderContext + public void setNiFiProperties(NiFiProperties properties) { + this.properties = properties; + } + + private InetSocketAddress getServiceAddress() { + final DiscoverableService service = serviceLocator.getService(); + if (service != null) { + return service.getServiceAddress(); + } + return null; + } + + private boolean isException(final Class exception, final ProtocolMessage protocolMessage) { + if (protocolMessage.wasException()) { + return exception.getName().equals(protocolMessage.getExceptionClass()); + } else { + return false; + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java new file mode 100644 index 0000000000..38d0dd861e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/DoesDnExistMessage.java @@ -0,0 +1,56 @@ +/* + * 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.cluster.authorization.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.cluster.authorization.protocol.message.ProtocolMessage.MessageType; + +/** + * @author unattributed + */ +@XmlRootElement(name = "doesDnExistMessage") +public class DoesDnExistMessage extends ProtocolMessage { + + private String dn; + + private boolean response; + + public DoesDnExistMessage() { + } + + @Override + public MessageType getType() { + return MessageType.DOES_DN_EXIST; + } + + public String getDn() { + return dn; + } + + public void setDn(String dn) { + this.dn = dn; + } + + public boolean getResponse() { + return response; + } + + public void setResponse(boolean response) { + this.response = response; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java new file mode 100644 index 0000000000..347163f184 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetAuthoritiesMessage.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.authorization.protocol.message; + +import java.util.HashSet; +import java.util.Set; +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.authorization.Authority; + +/** + * @author unattributed + */ +@XmlRootElement(name = "getAuthoritiesMessage") +public class GetAuthoritiesMessage extends ProtocolMessage { + + private String dn; + + private Set response = new HashSet<>(); + + public GetAuthoritiesMessage() { + } + + @Override + public MessageType getType() { + return MessageType.GET_AUTHORITIES; + } + + public String getDn() { + return dn; + } + + public void setDn(String dn) { + this.dn = dn; + } + + public Set getResponse() { + return response; + } + + public void setResponse(Set response) { + this.response = response; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java new file mode 100644 index 0000000000..717f2444e2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/GetGroupForUserMessage.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.authorization.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * @author unattributed + */ +@XmlRootElement(name = "getGroupForUserMessage") +public class GetGroupForUserMessage extends ProtocolMessage { + + private String dn; + + private String response; + + public GetGroupForUserMessage() { + } + + @Override + public MessageType getType() { + return MessageType.GET_GROUP_FOR_USER; + } + + public String getDn() { + return dn; + } + + public void setDn(String dn) { + this.dn = dn; + } + + public String getResponse() { + return response; + } + + public void setResponse(String response) { + this.response = response; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.java new file mode 100644 index 0000000000..102142aa60 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/ProtocolMessage.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.cluster.authorization.protocol.message; + +/** + * @author unattributed + */ +public abstract class ProtocolMessage { + + private String exceptionClass; + private String exceptionMessage; + + public static enum MessageType { + + DOES_DN_EXIST, + GET_AUTHORITIES, + GET_USERS, + GET_GROUP_FOR_USER + } + + public abstract MessageType getType(); + + public boolean wasException() { + return exceptionClass != null; + } + + public String getExceptionMessage() { + return exceptionMessage; + } + + public void setExceptionMessage(final String exceptionMessage) { + this.exceptionMessage = exceptionMessage; + } + + public String getExceptionClass() { + return exceptionClass; + } + + public void setExceptionClass(String exceptionClass) { + this.exceptionClass = exceptionClass; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java new file mode 100644 index 0000000000..97a1bc7f0b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/JaxbProtocolUtils.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.authorization.protocol.message.jaxb; + +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; + +/** + * @author unattributed + */ +public final class JaxbProtocolUtils { + + public static final String JAXB_CONTEXT_PATH = ObjectFactory.class.getPackage().getName(); + + public static final JAXBContext JAXB_CONTEXT = initializeJaxbContext(); + + /** + * Load the JAXBContext version. + */ + private static JAXBContext initializeJaxbContext() { + try { + return JAXBContext.newInstance(JAXB_CONTEXT_PATH); + } catch (JAXBException e) { + throw new RuntimeException("Unable to create JAXBContext."); + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java new file mode 100644 index 0000000000..5cde335919 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/java/org/apache/nifi/cluster/authorization/protocol/message/jaxb/ObjectFactory.java @@ -0,0 +1,45 @@ +/* + * 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.cluster.authorization.protocol.message.jaxb; + +import javax.xml.bind.annotation.XmlRegistry; +import org.apache.nifi.cluster.authorization.protocol.message.DoesDnExistMessage; +import org.apache.nifi.cluster.authorization.protocol.message.GetAuthoritiesMessage; +import org.apache.nifi.cluster.authorization.protocol.message.GetGroupForUserMessage; + +/** + * @author unattributed + */ +@XmlRegistry +public class ObjectFactory { + + public ObjectFactory() { + } + + public DoesDnExistMessage createDoesDnExistMessage() { + return new DoesDnExistMessage(); + } + + public GetAuthoritiesMessage createGetAuthoritiesMessage() { + return new GetAuthoritiesMessage(); + } + + public GetGroupForUserMessage createGetGroupForUserMessage() { + return new GetGroupForUserMessage(); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider new file mode 100644 index 0000000000..56f4c3e32f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-authorization-provider/src/main/resources/META-INF/services/org.apache.nifi.authorization.AuthorityProvider @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +org.apache.nifi.cluster.authorization.ClusterManagerAuthorizationProvider +org.apache.nifi.cluster.authorization.NodeAuthorizationProvider \ No newline at end of file diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore b/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml b/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml new file mode 100644 index 0000000000..535108548d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/pom.xml @@ -0,0 +1,69 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + framework-cluster-protocol + jar + NiFi Framework Cluster Protocol + The messaging protocol for clustered NiFi + + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-properties + + + org.apache.nifi + nifi-logging-utils + + + org.apache.nifi + nifi-socket-utils + + + org.apache.nifi + nifi-security + + + org.apache.nifi + core-api + + + + + org.springframework + spring-core + + + org.springframework + spring-beans + + + org.springframework + spring-context + + + diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java new file mode 100644 index 0000000000..fa1547ffcb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ClusterManagerProtocolSender.java @@ -0,0 +1,69 @@ +/* + * 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.cluster.protocol; + +import org.apache.nifi.cluster.protocol.message.DisconnectMessage; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; +import org.apache.nifi.reporting.BulletinRepository; + +/** + * An interface for sending protocol messages from the cluster manager to nodes. + * + * @author unattributed + */ +public interface ClusterManagerProtocolSender { + + /** + * Sends a "flow request" message to a node. + * @param msg a message + * @return the response + * @throws ProtocolException if communication failed + */ + FlowResponseMessage requestFlow(FlowRequestMessage msg) throws ProtocolException; + + /** + * Sends a "reconnection request" message to a node. + * @param msg a message + * @return + * @throws ProtocolException if communication failed + */ + ReconnectionResponseMessage requestReconnection(ReconnectionRequestMessage msg) throws ProtocolException; + + /** + * Sends a "disconnection request" message to a node. + * @param msg a message + * @throws ProtocolException if communication failed + */ + void disconnect(DisconnectMessage msg) throws ProtocolException; + + /** + * Sends an "assign primary role" message to a node. + * @param msg a message + * @throws ProtocolException if communication failed + */ + void assignPrimaryRole(PrimaryRoleAssignmentMessage msg) throws ProtocolException; + + /** + * Sets the {@link BulletinRepository} that can be used to report bulletins + * @param bulletinRepository + */ + void setBulletinRepository(final BulletinRepository bulletinRepository); +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java new file mode 100644 index 0000000000..1b5d007e78 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionRequest.java @@ -0,0 +1,44 @@ +/* + * 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.cluster.protocol; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionRequestAdapter; + +/** + * A node's request to connect to the cluster. The request contains a proposed + * identifier. + * + * @author unattributed + */ +@XmlJavaTypeAdapter(ConnectionRequestAdapter.class) +public class ConnectionRequest { + + private final NodeIdentifier proposedNodeIdentifier; + + public ConnectionRequest(final NodeIdentifier proposedNodeIdentifier) { + if(proposedNodeIdentifier == null) { + throw new IllegalArgumentException("Proposed node identifier may not be null."); + } + this.proposedNodeIdentifier = proposedNodeIdentifier; + } + + public NodeIdentifier getProposedNodeIdentifier() { + return proposedNodeIdentifier; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java new file mode 100644 index 0000000000..7a5ff2b321 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ConnectionResponse.java @@ -0,0 +1,141 @@ +/* + * 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.cluster.protocol; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.cluster.protocol.jaxb.message.ConnectionResponseAdapter; + +/** + * The cluster manager's response to a node's connection request. If the manager + * has a current copy of the data flow, then it is returned with a node identifier + * to the node. Otherwise, the manager will provide a "try again in X seconds" + * response to the node in hopes that a current data flow will be available upon + * subsequent requests. + * + * @author unattributed + */ +@XmlJavaTypeAdapter(ConnectionResponseAdapter.class) +public class ConnectionResponse { + + private final boolean blockedByFirewall; + private final int tryLaterSeconds; + private final NodeIdentifier nodeIdentifier; + private final StandardDataFlow dataFlow; + private final boolean primary; + private final Integer managerRemoteInputPort; + private final Boolean managerRemoteCommsSecure; + private final String instanceId; + + private volatile String clusterManagerDN; + + public ConnectionResponse(final NodeIdentifier nodeIdentifier, final StandardDataFlow dataFlow, final boolean primary, + final Integer managerRemoteInputPort, final Boolean managerRemoteCommsSecure, final String instanceId) { + if(nodeIdentifier == null) { + throw new IllegalArgumentException("Node identifier may not be empty or null."); + } else if(dataFlow == null) { + throw new IllegalArgumentException("DataFlow may not be null."); + } + this.nodeIdentifier = nodeIdentifier; + this.dataFlow = dataFlow; + this.tryLaterSeconds = 0; + this.blockedByFirewall = false; + this.primary = primary; + this.managerRemoteInputPort = managerRemoteInputPort; + this.managerRemoteCommsSecure = managerRemoteCommsSecure; + this.instanceId = instanceId; + } + + public ConnectionResponse(final int tryLaterSeconds) { + if(tryLaterSeconds <= 0) { + throw new IllegalArgumentException("Try-Later seconds may not be nonnegative: " + tryLaterSeconds); + } + this.dataFlow = null; + this.nodeIdentifier = null; + this.tryLaterSeconds = tryLaterSeconds; + this.blockedByFirewall = false; + this.primary = false; + this.managerRemoteInputPort = null; + this.managerRemoteCommsSecure = null; + this.instanceId = null; + } + + private ConnectionResponse() { + this.dataFlow = null; + this.nodeIdentifier = null; + this.tryLaterSeconds = 0; + this.blockedByFirewall = true; + this.primary = false; + this.managerRemoteInputPort = null; + this.managerRemoteCommsSecure = null; + this.instanceId = null; + } + + public static ConnectionResponse createBlockedByFirewallResponse() { + return new ConnectionResponse(); + } + + public boolean isPrimary() { + return primary; + } + + public boolean shouldTryLater() { + return tryLaterSeconds > 0; + } + + public boolean isBlockedByFirewall() { + return blockedByFirewall; + } + + public int getTryLaterSeconds() { + return tryLaterSeconds; + } + + public StandardDataFlow getDataFlow() { + return dataFlow; + } + + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public Integer getManagerRemoteInputPort() { + return managerRemoteInputPort; + } + + public Boolean isManagerRemoteCommsSecure() { + return managerRemoteCommsSecure; + } + + public String getInstanceId() { + return instanceId; + } + + public void setClusterManagerDN(final String dn) { + this.clusterManagerDN = dn; + } + + /** + * Returns the DN of the NCM, if it is available or null otherwise. + * + * @return + */ + public String getClusterManagerDN() { + return clusterManagerDN; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java new file mode 100644 index 0000000000..67324a1e2e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/Heartbeat.java @@ -0,0 +1,68 @@ +/* + * 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.cluster.protocol; + +import java.util.Date; +import javax.xml.bind.annotation.XmlTransient; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.jaxb.message.HeartbeatAdapter; + +/** + * A heartbeat for indicating the status of a node to the cluster. + * @author unattributed + */ +@XmlJavaTypeAdapter(HeartbeatAdapter.class) +public class Heartbeat { + + private final NodeIdentifier nodeIdentifier; + private final boolean primary; + private final boolean connected; + private final long createdTimestamp; + private final byte[] payload; + + public Heartbeat(final NodeIdentifier nodeIdentifier, final boolean primary, final boolean connected, final byte[] payload) { + if(nodeIdentifier == null) { + throw new IllegalArgumentException("Node Identifier may not be null."); + } + this.nodeIdentifier = nodeIdentifier; + this.primary = primary; + this.connected = connected; + this.payload = payload; + this.createdTimestamp = new Date().getTime(); + } + + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public byte[] getPayload() { + return payload; + } + + public boolean isPrimary() { + return primary; + } + + public boolean isConnected() { + return connected; + } + + @XmlTransient + public long getCreatedTimestamp() { + return createdTimestamp; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java new file mode 100644 index 0000000000..a12052459c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeBulletins.java @@ -0,0 +1,44 @@ +/* + * 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.cluster.protocol; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeBulletinsAdapter; + +/** + * + */ +@XmlJavaTypeAdapter(NodeBulletinsAdapter.class) +public class NodeBulletins { + + private final NodeIdentifier nodeIdentifier; + private final byte[] payload; + + public NodeBulletins(NodeIdentifier nodeIdentifier, byte[] payload) { + this.nodeIdentifier = nodeIdentifier; + this.payload = payload; + } + + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public byte[] getPayload() { + return payload; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java new file mode 100644 index 0000000000..18931860c9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeIdentifier.java @@ -0,0 +1,172 @@ +/* + * 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.cluster.protocol; + +import org.apache.commons.lang3.StringUtils; + +/** + * A node identifier denoting the coordinates of a flow controller that is connected + * to a cluster. Nodes provide an external public API interface and an internal private + * interface for communicating with the cluster. + * + * The external API interface and internal protocol each require an IP or hostname + * as well as a port for communicating. + * + * This class overrides hashCode and equals and considers two instances to be + * equal if they have the equal IDs. + * + * @author unattributed + * @Immutable + * @Threadsafe + */ +public class NodeIdentifier { + + /** the unique identifier for the node */ + private final String id; + + /** the IP or hostname to use for sending requests to the node's external interface */ + private final String apiAddress; + + /** the port to use use for sending requests to the node's external interface */ + private final int apiPort; + + /** the IP or hostname to use for sending requests to the node's internal interface */ + private final String socketAddress; + + /** the port to use use for sending requests to the node's internal interface */ + private final int socketPort; + + private final String nodeDn; + + public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort) { + this(id, apiAddress, apiPort, socketAddress, socketPort, null); + } + + public NodeIdentifier(final String id, final String apiAddress, final int apiPort, final String socketAddress, final int socketPort, final String dn) { + + if(StringUtils.isBlank(id)) { + throw new IllegalArgumentException("Node ID may not be empty or null."); + } else if(StringUtils.isBlank(apiAddress)) { + throw new IllegalArgumentException("Node API address may not be empty or null."); + } else if(StringUtils.isBlank(socketAddress)) { + throw new IllegalArgumentException("Node socket address may not be empty or null."); + } + + validatePort(apiPort); + validatePort(socketPort); + + this.id = id; + this.apiAddress = apiAddress; + this.apiPort = apiPort; + this.socketAddress = socketAddress; + this.socketPort = socketPort; + this.nodeDn = dn; + } + + public String getId() { + return id; + } + + public String getDN() { + return nodeDn; + } + + public String getApiAddress() { + return apiAddress; + } + + public int getApiPort() { + return apiPort; + } + + public String getSocketAddress() { + return socketAddress; + } + + public int getSocketPort() { + return socketPort; + } + + private void validatePort(final int port) { + if(port < 1 || port > 65535) { + throw new IllegalArgumentException("Port must be inclusively in the range [1, 65535]. Port given: " + port); + } + } + + /** + * Compares the id of two node identifiers for equality. + * + * @param obj a node identifier + * + * @return true if the id is equal; false otherwise + */ + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final NodeIdentifier other = (NodeIdentifier) obj; + if ((this.id == null) ? (other.id != null) : !this.id.equals(other.id)) { + return false; + } + return true; + } + + /** + * Compares API address/port and socket address/port for equality. The + * id is not used for comparison. + * + * @param other a node identifier + * + * @return true if API address/port and socket address/port are equal; false + * otherwise + */ + public boolean logicallyEquals(final NodeIdentifier other) { + if(other == null) { + return false; + } + if ((this.apiAddress == null) ? (other.apiAddress != null) : !this.apiAddress.equals(other.apiAddress)) { + return false; + } + if(this.apiPort != other.apiPort) { + return false; + } + if ((this.socketAddress == null) ? (other.socketAddress != null) : !this.socketAddress.equals(other.socketAddress)) { + return false; + } + if(this.socketPort != other.socketPort) { + return false; + } + return true; + } + + @Override + public int hashCode() { + int hash = 7; + hash = 31 * hash + (this.id != null ? this.id.hashCode() : 0); + return hash; + } + + @Override + public String toString() { + return "[" + "id=" + id + ", apiAddress=" + apiAddress + ", apiPort=" + apiPort + ", socketAddress=" + socketAddress + ", socketPort=" + socketPort + ']'; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java new file mode 100644 index 0000000000..1edcb91840 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/NodeProtocolSender.java @@ -0,0 +1,73 @@ +/* + * 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.cluster.protocol; + +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage; + +/** + * An interface for sending protocol messages from a node to the cluster manager. + * @author unattributed + */ +public interface NodeProtocolSender { + + /** + * Sends a "connection request" message to the cluster manager. + * @param msg a message + * @return the response + * @throws UnknownServiceAddressException if the cluster manager's address is not known + * @throws ProtocolException if communication failed + */ + ConnectionResponseMessage requestConnection(ConnectionRequestMessage msg) throws ProtocolException, UnknownServiceAddressException; + + /** + * Sends a "heartbeat" message to the cluster manager. + * @param msg a message + * @throws UnknownServiceAddressException if the cluster manager's address is not known + * @throws ProtocolException if communication failed + */ + void heartbeat(HeartbeatMessage msg) throws ProtocolException, UnknownServiceAddressException; + + /** + * Sends a bulletins message to the cluster manager. + * @param msg + * @throws ProtocolException + * @throws UnknownServiceAddressException + */ + void sendBulletins(NodeBulletinsMessage msg) throws ProtocolException, UnknownServiceAddressException; + + /** + * Sends a failure notification if the controller was unable start. + * @param msg a message + * @throws UnknownServiceAddressException if the cluster manager's address is not known + * @throws ProtocolException if communication failed + */ + void notifyControllerStartupFailure(ControllerStartupFailureMessage msg) throws ProtocolException, UnknownServiceAddressException; + + /** + * Sends a failure notification if the node was unable to reconnect to the cluster + * @param msg a message + * @throws UnknownServiceAddressException if the cluster manager's address is not known + * @throws ProtocolException if communication failed + */ + void notifyReconnectionFailure(ReconnectionFailureMessage msg) throws ProtocolException, UnknownServiceAddressException; + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java new file mode 100644 index 0000000000..b614e7630e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolContext.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol; + +/** + * The context for communicating using the internal cluster protocol. + * + * @param The type of protocol message. + * + * @author unattributed + */ +public interface ProtocolContext { + + /** + * Creates a marshaller for serializing protocol messages. + * @return a marshaller + */ + ProtocolMessageMarshaller createMarshaller(); + + /** + * Creates an unmarshaller for deserializing protocol messages. + * @return a unmarshaller + */ + ProtocolMessageUnmarshaller createUnmarshaller(); +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java new file mode 100644 index 0000000000..f11ad84aa0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolException.java @@ -0,0 +1,40 @@ +/* + * 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.cluster.protocol; + +/** + * The base exception for problems encountered while communicating within the + * cluster. + * @author unattributed + */ +public class ProtocolException extends RuntimeException { + + public ProtocolException() { + } + + public ProtocolException(String msg) { + super(msg); + } + + public ProtocolException(Throwable cause) { + super(cause); + } + + public ProtocolException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java new file mode 100644 index 0000000000..6de87db997 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolHandler.java @@ -0,0 +1,44 @@ +/* + * 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.cluster.protocol; + +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; + +/** + * A handler for processing protocol messages. + * @author unattributed + */ +public interface ProtocolHandler { + + /** + * Handles the given protocol message or throws an exception if it cannot + * handle the message. If no response is needed by the protocol, then null + * should be returned. + * + * @param msg a message + * @return a response or null, if no response is necessary + * + * @throws ProtocolException if the message could not be processed + */ + ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException; + + /** + * @param msg + * @return true if the handler can process the given message; false otherwise + */ + boolean canHandle(ProtocolMessage msg); +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java new file mode 100644 index 0000000000..32f0f5ddfe --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolListener.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.nifi.reporting.BulletinRepository; + +/** + * Defines the interface for a listener to process protocol messages. + * @author unattributed + */ +public interface ProtocolListener { + + /** + * Starts the instance for listening for messages. Start may only be called + * if the instance is not running. + * @throws java.io.IOException + */ + void start() throws IOException; + + /** + * Stops the instance from listening for messages. Stop may only be called + * if the instance is running. + * @throws java.io.IOException + */ + void stop() throws IOException; + + /** + * @return true if the instance is started; false otherwise. + */ + boolean isRunning(); + + /** + * @return the handlers registered with the listener + */ + Collection getHandlers(); + + /** + * Registers a handler with the listener. + * @param handler a handler + */ + void addHandler(ProtocolHandler handler); + + /** + * Sets the BulletinRepository that can be used to report bulletins + * @param bulletinRepository + */ + void setBulletinRepository(BulletinRepository bulletinRepository); + + /** + * Unregisters the handler with the listener. + * @param handler a handler + * @return true if the handler was removed; false otherwise + */ + boolean removeHandler(ProtocolHandler handler); +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java new file mode 100644 index 0000000000..bb436e0c4e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageMarshaller.java @@ -0,0 +1,38 @@ +/* + * 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.cluster.protocol; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * Defines a marshaller for serializing protocol messages. + * + * @param The type of protocol message. + * + * @author unattributed + */ +public interface ProtocolMessageMarshaller { + + /** + * Serializes the given message to the given output stream. + * @param msg a message + * @param os an output stream + * @throws IOException if the message could not be serialized to the stream + */ + void marshal(T msg, OutputStream os) throws IOException; +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java new file mode 100644 index 0000000000..c690e7bfe7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/ProtocolMessageUnmarshaller.java @@ -0,0 +1,38 @@ +/* + * 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.cluster.protocol; + +import java.io.IOException; +import java.io.InputStream; + +/** + * Defines an unmarshaller for deserializing protocol messages. + * + * @param The type of protocol message. + * + * @author unattributed + */ +public interface ProtocolMessageUnmarshaller { + + /** + * Deserializes a message on the given input stream. + * @param is an input stream + * @return + * @throws IOException if the message could not be deserialized from the stream + */ + T unmarshal(InputStream is) throws IOException; +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java new file mode 100644 index 0000000000..c2d16fc01b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/StandardDataFlow.java @@ -0,0 +1,105 @@ +/* + * 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.cluster.protocol; + +import org.apache.nifi.cluster.protocol.DataFlow; +import java.io.Serializable; +import java.util.Arrays; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.cluster.protocol.jaxb.message.DataFlowAdapter; + +/** + * Represents a dataflow, which includes the raw bytes of the flow.xml and + * whether processors should be started automatically at application startup. + */ +@XmlJavaTypeAdapter(DataFlowAdapter.class) +public class StandardDataFlow implements Serializable, DataFlow { + + private final byte[] flow; + private final byte[] templateBytes; + private final byte[] snippetBytes; + + private boolean autoStartProcessors; + + /** + * Constructs an instance. + * + * @param flow a valid flow as bytes, which cannot be null + * @param templateBytes an XML representation of templates + * @param snippetBytes an XML representation of snippets + * + * @throws NullPointerException if any argument is null + */ + public StandardDataFlow(final byte[] flow, final byte[] templateBytes, final byte[] snippetBytes) { + this.flow = flow; + this.templateBytes = templateBytes; + this.snippetBytes = snippetBytes; + } + + public StandardDataFlow(final DataFlow toCopy) { + this.flow = copy(toCopy.getFlow()); + this.templateBytes = copy(toCopy.getTemplates()); + this.snippetBytes = copy(toCopy.getSnippets()); + this.autoStartProcessors = toCopy.isAutoStartProcessors(); + } + + private static byte[] copy(final byte[] bytes) { + return bytes == null ? null : Arrays.copyOf(bytes, bytes.length); + } + + /** + * @return the raw byte array of the flow + */ + public byte[] getFlow() { + return flow; + } + + /** + * @return the raw byte array of the templates + */ + public byte[] getTemplates() { + return templateBytes; + } + + /** + * @return the raw byte array of the snippets + */ + public byte[] getSnippets() { + return snippetBytes; + } + + /** + * @return true if processors should be automatically started at application + * startup; false otherwise + */ + public boolean isAutoStartProcessors() { + return autoStartProcessors; + } + + /** + * + * Sets the flag to automatically start processors at application startup. + * + * @param autoStartProcessors true if processors should be automatically + * started at application startup; false otherwise + */ + public void setAutoStartProcessors(final boolean autoStartProcessors) { + this.autoStartProcessors = autoStartProcessors; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java new file mode 100644 index 0000000000..41c74eb10b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/UnknownServiceAddressException.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol; + +/** + * Represents the exceptional case when a service's address is not known. + * @author unattributed + */ +public class UnknownServiceAddressException extends RuntimeException { + + public UnknownServiceAddressException() { + } + + public UnknownServiceAddressException(String msg) { + super(msg); + } + + public UnknownServiceAddressException(Throwable cause) { + super(cause); + } + + public UnknownServiceAddressException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java new file mode 100644 index 0000000000..ceb3fcbf9e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImpl.java @@ -0,0 +1,245 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketException; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.message.DisconnectMessage; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.io.socket.SocketUtils; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.util.FormatUtils; + +/** + * A protocol sender for sending protocol messages from the cluster manager to + * nodes. + * + * Connection-type requests (e.g., reconnection, disconnection) by nature of + * starting/stopping flow controllers take longer than other types of protocol + * messages. Therefore, a handshake timeout may be specified to lengthen the + * allowable time for communication with the node. + * + * @author unattributed + */ +public class ClusterManagerProtocolSenderImpl implements ClusterManagerProtocolSender { + + + private final ProtocolContext protocolContext; + private final SocketConfiguration socketConfiguration; + private int handshakeTimeoutSeconds; + private volatile BulletinRepository bulletinRepository; + + public ClusterManagerProtocolSenderImpl(final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) { + if(socketConfiguration == null) { + throw new IllegalArgumentException("Socket configuration may not be null."); + } else if(protocolContext == null) { + throw new IllegalArgumentException("Protocol Context may not be null."); + } + this.socketConfiguration = socketConfiguration; + this.protocolContext = protocolContext; + this.handshakeTimeoutSeconds = -1; // less than zero denotes variable not configured + } + + @Override + public void setBulletinRepository(final BulletinRepository bulletinRepository) { + this.bulletinRepository = bulletinRepository; + } + + /** + * Requests the data flow from a node. + * @param msg a message + * @return the message response + * @throws @throws ProtocolException if the message failed to be sent or the response was malformed + */ + @Override + public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException { + Socket socket = null; + try { + socket = createSocket(msg.getNodeId(), false); + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + + final ProtocolMessage response; + try { + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); + response = unmarshaller.unmarshal(socket.getInputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed unmarshalling '" + MessageType.FLOW_RESPONSE + "' protocol message due to: " + ioe, ioe); + } + + if(MessageType.FLOW_RESPONSE == response.getType()) { + return (FlowResponseMessage) response; + } else { + throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'"); + } + + } finally { + SocketUtils.closeQuietly(socket); + } + } + + /** + * Requests a node to reconnect to the cluster. The configured value for + * handshake timeout is applied to the socket before making the request. + * @param msg a message + * @return the response + * @throws ProtocolException if the message failed to be sent or the response was malformed + */ + @Override + public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException { + Socket socket = null; + try { + socket = createSocket(msg.getNodeId(), true); + + // marshal message to output stream + try { + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + + + final ProtocolMessage response; + try { + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); + response = unmarshaller.unmarshal(socket.getInputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed unmarshalling '" + MessageType.RECONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe); + } + + if(MessageType.RECONNECTION_RESPONSE == response.getType()) { + return (ReconnectionResponseMessage) response; + } else { + throw new ProtocolException("Expected message type '" + MessageType.FLOW_RESPONSE + "' but found '" + response.getType() + "'"); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + + /** + * Requests a node to disconnect from the cluster. The configured value for + * handshake timeout is applied to the socket before making the request. + * @param msg a message + * @throws ProtocolException if the message failed to be sent + */ + @Override + public void disconnect(final DisconnectMessage msg) throws ProtocolException { + Socket socket = null; + try { + socket = createSocket(msg.getNodeId(), true); + + // marshal message to output stream + try { + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + + /** + * Assigns the primary role to a node. + * + * @param msg a message + * + * @throws ProtocolException if the message failed to be sent + */ + @Override + public void assignPrimaryRole(final PrimaryRoleAssignmentMessage msg) throws ProtocolException { + Socket socket = null; + try { + socket = createSocket(msg.getNodeId(), true); + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + + + private void setConnectionHandshakeTimeoutOnSocket(final Socket socket) throws SocketException { + // update socket timeout, if handshake timeout was set; otherwise use socket's current timeout + if(handshakeTimeoutSeconds >= 0) { + socket.setSoTimeout(handshakeTimeoutSeconds * 1000); + } + } + + public SocketConfiguration getSocketConfiguration() { + return socketConfiguration; + } + + public int getHandshakeTimeoutSeconds() { + return handshakeTimeoutSeconds; + } + + public void setHandshakeTimeout(final String handshakeTimeout) { + this.handshakeTimeoutSeconds = (int) FormatUtils.getTimeDuration(handshakeTimeout, TimeUnit.SECONDS); + } + + private Socket createSocket(final NodeIdentifier nodeId, final boolean applyHandshakeTimeout) { + return createSocket(nodeId.getSocketAddress(), nodeId.getSocketPort(), applyHandshakeTimeout); + } + + private Socket createSocket(final String host, final int port, final boolean applyHandshakeTimeout) { + try { + // create a socket + final Socket socket = SocketUtils.createSocket(InetSocketAddress.createUnresolved(host, port), socketConfiguration); + if ( applyHandshakeTimeout ) { + setConnectionHandshakeTimeoutOnSocket(socket); + } + return socket; + } catch(final IOException ioe) { + throw new ProtocolException("Failed to create socket due to: " + ioe, ioe); + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java new file mode 100644 index 0000000000..933e5fade8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderListener.java @@ -0,0 +1,118 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolListener; +import org.apache.nifi.cluster.protocol.message.DisconnectMessage; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; +import org.apache.nifi.reporting.BulletinRepository; + +/** + * A wrapper class for consolidating a protocol sender and listener for the cluster + * manager. + * + * @author unattributed + */ +public class ClusterManagerProtocolSenderListener implements ClusterManagerProtocolSender, ProtocolListener { + + private final ClusterManagerProtocolSender sender; + + private final ProtocolListener listener; + + public ClusterManagerProtocolSenderListener(final ClusterManagerProtocolSender sender, final ProtocolListener listener) { + if(sender == null) { + throw new IllegalArgumentException("ClusterManagerProtocolSender may not be null."); + } else if(listener == null) { + throw new IllegalArgumentException("ProtocolListener may not be null."); + } + this.sender = sender; + this.listener = listener; + } + + @Override + public void stop() throws IOException { + if(!isRunning()) { + throw new IllegalStateException("Instance is already stopped."); + } + listener.stop(); + } + + @Override + public void start() throws IOException { + if(isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + listener.start(); + } + + @Override + public boolean isRunning() { + return listener.isRunning(); + } + + @Override + public boolean removeHandler(final ProtocolHandler handler) { + return listener.removeHandler(handler); + } + + @Override + public Collection getHandlers() { + return listener.getHandlers(); + } + + @Override + public void addHandler(final ProtocolHandler handler) { + listener.addHandler(handler); + } + + @Override + public void setBulletinRepository(final BulletinRepository bulletinRepository) { + listener.setBulletinRepository(bulletinRepository); + sender.setBulletinRepository(bulletinRepository); + } + + @Override + public FlowResponseMessage requestFlow(final FlowRequestMessage msg) throws ProtocolException { + return sender.requestFlow(msg); + } + + @Override + public ReconnectionResponseMessage requestReconnection(final ReconnectionRequestMessage msg) throws ProtocolException { + return sender.requestReconnection(msg); + } + + @Override + public void disconnect(DisconnectMessage msg) throws ProtocolException { + sender.disconnect(msg); + } + + @Override + public void assignPrimaryRole(PrimaryRoleAssignmentMessage msg) throws ProtocolException { + sender.assignPrimaryRole(msg); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java new file mode 100644 index 0000000000..24e51e0ee9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscovery.java @@ -0,0 +1,181 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Collection; +import java.util.Collections; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastServiceDiscovery; +import org.apache.nifi.reporting.BulletinRepository; + +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolListener; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; +import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An implementation for discovering services by way of "service broadcast" type + * protocol messages over multicast. + * + * The client caller is responsible for starting and stopping the service + * discovery. The instance must be stopped before termination of the JVM to + * ensure proper resource clean-up. + * + * @author unattributed + */ +public class ClusterServiceDiscovery implements MulticastServiceDiscovery, ProtocolListener { + + private static final Logger logger = LoggerFactory.getLogger(ClusterServiceDiscovery.class); + private final String serviceName; + private final MulticastConfiguration multicastConfiguration; + private final MulticastProtocolListener listener; + private volatile BulletinRepository bulletinRepository; + + /* + * guarded by this + */ + private DiscoverableService service; + + + public ClusterServiceDiscovery(final String serviceName, final InetSocketAddress multicastAddress, + final MulticastConfiguration multicastConfiguration, final ProtocolContext protocolContext) { + + if (StringUtils.isBlank(serviceName)) { + throw new IllegalArgumentException("Service name may not be null or empty."); + } else if (multicastAddress == null) { + throw new IllegalArgumentException("Multicast address may not be null."); + } else if (multicastAddress.getAddress().isMulticastAddress() == false) { + throw new IllegalArgumentException("Multicast group must be a Class D address."); + } else if (protocolContext == null) { + throw new IllegalArgumentException("Protocol Context may not be null."); + } else if (multicastConfiguration == null) { + throw new IllegalArgumentException("Multicast configuration may not be null."); + } + + this.serviceName = serviceName; + this.multicastConfiguration = multicastConfiguration; + this.listener = new MulticastProtocolListener(1, multicastAddress, multicastConfiguration, protocolContext); + listener.addHandler(new ClusterManagerServiceBroadcastHandler()); + } + + @Override + public void setBulletinRepository(final BulletinRepository bulletinRepository) { + this.bulletinRepository = bulletinRepository; + } + + @Override + public synchronized DiscoverableService getService() { + return service; + } + + @Override + public InetSocketAddress getMulticastAddress() { + return listener.getMulticastAddress(); + } + + @Override + public Collection getHandlers() { + return Collections.unmodifiableCollection(listener.getHandlers()); + } + + @Override + public void addHandler(ProtocolHandler handler) { + listener.addHandler(handler); + } + + @Override + public boolean removeHandler(ProtocolHandler handler) { + return listener.removeHandler(handler); + } + + @Override + public boolean isRunning() { + return listener.isRunning(); + } + + @Override + public void start() throws IOException { + if (isRunning()) { + throw new IllegalStateException("Instance is already running."); + } + listener.start(); + } + + @Override + public void stop() throws IOException { + if (isRunning() == false) { + throw new IllegalStateException("Instance is already stopped."); + } + listener.stop(); + } + + public String getServiceName() { + return serviceName; + } + + public MulticastConfiguration getMulticastConfiguration() { + return multicastConfiguration; + } + + private class ClusterManagerServiceBroadcastHandler implements ProtocolHandler { + + @Override + public boolean canHandle(final ProtocolMessage msg) { + return MessageType.SERVICE_BROADCAST == msg.getType(); + } + + @Override + public ProtocolMessage handle(final ProtocolMessage msg) throws ProtocolException { + synchronized (ClusterServiceDiscovery.this) { + if (canHandle(msg) == false) { + throw new ProtocolException("Handler cannot handle message type: " + msg.getType()); + } else { + final ServiceBroadcastMessage broadcastMsg = (ServiceBroadcastMessage) msg; + if (serviceName.equals(broadcastMsg.getServiceName())) { + final DiscoverableService oldService = service; + if (oldService == null + || broadcastMsg.getAddress().equalsIgnoreCase(oldService.getServiceAddress().getHostName()) == false + || broadcastMsg.getPort() != oldService.getServiceAddress().getPort()) { + service = new DiscoverableServiceImpl(serviceName, InetSocketAddress.createUnresolved(broadcastMsg.getAddress(), broadcastMsg.getPort())); + final InetSocketAddress oldServiceAddress = (oldService == null) ? null : oldService.getServiceAddress(); + logger.info(String.format("Updating cluster service address for '%s' from '%s' to '%s'", serviceName, prettyPrint(oldServiceAddress), prettyPrint(service.getServiceAddress()))); + } + } + return null; + } + } + } + } + + private String prettyPrint(final InetSocketAddress address) { + if (address == null) { + return "0.0.0.0:0"; + } else { + return address.getHostName() + ":" + address.getPort(); + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java new file mode 100644 index 0000000000..bebfde8b2e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocator.java @@ -0,0 +1,229 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.io.socket.multicast.ServiceDiscovery; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements the ServiceLocator interface for locating the socket address + * of a cluster service. Depending on configuration, the address may be located + * using service discovery. If using service discovery, then the service methods + * must be used for starting and stopping discovery. + * + * Service discovery may be used in conjunction with a fixed port. In this case, + * the service discovery will yield the service IP/host while the fixed port will + * be used for the port. + * + * Alternatively, the instance may be configured with exact service location, in + * which case, no service discovery occurs and the caller will always receive the + * configured service. + * + * @author unattributed + */ +public class ClusterServiceLocator implements ServiceDiscovery { + + private static final Logger logger = LoggerFactory.getLogger(ClusterServiceLocator.class); + + private final String serviceName; + + private final ClusterServiceDiscovery serviceDiscovery; + + private final DiscoverableService fixedService; + + private final int fixedServicePort; + + private final AttemptsConfig attemptsConfig = new AttemptsConfig(); + + private final AtomicBoolean running = new AtomicBoolean(false); + + public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery) { + if(serviceDiscovery == null) { + throw new IllegalArgumentException("ClusterServiceDiscovery may not be null."); + } + this.serviceDiscovery = serviceDiscovery; + this.fixedService = null; + this.fixedServicePort = 0; + this.serviceName = serviceDiscovery.getServiceName(); + } + + public ClusterServiceLocator(final ClusterServiceDiscovery serviceDiscovery, final int fixedServicePort) { + if(serviceDiscovery == null) { + throw new IllegalArgumentException("ClusterServiceDiscovery may not be null."); + } + this.serviceDiscovery = serviceDiscovery; + this.fixedService = null; + this.fixedServicePort = fixedServicePort; + this.serviceName = serviceDiscovery.getServiceName(); + } + + public ClusterServiceLocator(final DiscoverableService fixedService) { + if(fixedService == null) { + throw new IllegalArgumentException("Service may not be null."); + } + this.serviceDiscovery = null; + this.fixedService = fixedService; + this.fixedServicePort = 0; + this.serviceName = fixedService.getServiceName(); + } + + @Override + public DiscoverableService getService() { + + final int numAttemptsValue; + final int secondsBetweenAttempts; + synchronized(this) { + numAttemptsValue = attemptsConfig.numAttempts; + secondsBetweenAttempts = attemptsConfig.getTimeBetweenAttempts(); + } + + // try for a configured amount of attempts to retrieve the service address + for(int i = 0; i < numAttemptsValue; i++) { + + if(fixedService != null) { + return fixedService; + } else if(serviceDiscovery != null) { + + final DiscoverableService discoveredService = serviceDiscovery.getService(); + + // if we received an address + if(discoveredService != null) { + // if we were configured with a fixed port, then use the discovered host and fixed port; otherwise use the discovered address + if(fixedServicePort > 0) { + // create service using discovered service name and address with fixed service port + final InetSocketAddress addr = InetSocketAddress.createUnresolved(discoveredService.getServiceAddress().getHostName(), fixedServicePort); + final DiscoverableService result = new DiscoverableServiceImpl(discoveredService.getServiceName(), addr); + return result; + } else { + return discoveredService; + } + } + } + + // could not obtain service address, so sleep a bit + try { + logger.debug(String.format("Locating Cluster Service '%s' Attempt: %d of %d failed. Trying again in %d seconds.", + serviceName, (i + 1), numAttemptsValue, secondsBetweenAttempts)); + Thread.sleep(secondsBetweenAttempts * 1000); + } catch(final InterruptedException ie) { + break; + } + + } + + return null; + } + + public boolean isRunning() { + if(serviceDiscovery != null) { + return serviceDiscovery.isRunning(); + } else { + return running.get(); + } + } + + public void start() throws IOException { + + if(isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + + if(serviceDiscovery != null) { + serviceDiscovery.start(); + } + running.set(true); + } + + public void stop() throws IOException { + + if(isRunning() == false) { + throw new IllegalStateException("Instance is already stopped."); + } + + if(serviceDiscovery != null) { + serviceDiscovery.stop(); + } + running.set(false); + } + + public synchronized void setAttemptsConfig(final AttemptsConfig config) { + if(config == null) { + throw new IllegalArgumentException("Attempts configuration may not be null."); + } + this.attemptsConfig.numAttempts = config.numAttempts; + this.attemptsConfig.timeBetweenAttempts = config.timeBetweenAttempts; + this.attemptsConfig.timeBetweenAttempsUnit = config.timeBetweenAttempsUnit; + } + + public synchronized AttemptsConfig getAttemptsConfig() { + final AttemptsConfig config = new AttemptsConfig(); + config.numAttempts = this.attemptsConfig.numAttempts; + config.timeBetweenAttempts = this.attemptsConfig.timeBetweenAttempts; + config.timeBetweenAttempsUnit = this.attemptsConfig.timeBetweenAttempsUnit; + return config; + } + + public static class AttemptsConfig { + + private int numAttempts = 1; + + private int timeBetweenAttempts = 1; + + private TimeUnit timeBetweenAttempsUnit = TimeUnit.SECONDS; + + public int getNumAttempts() { + return numAttempts; + } + + public void setNumAttempts(int numAttempts) { + if(numAttempts <= 0) { + throw new IllegalArgumentException("Number of attempts must be positive: " + numAttempts); + } + this.numAttempts = numAttempts; + } + + public TimeUnit getTimeBetweenAttemptsUnit() { + return timeBetweenAttempsUnit; + } + + public void setTimeBetweenAttempsUnit(TimeUnit timeBetweenAttempsUnit) { + if(timeBetweenAttempts <= 0) { + throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts); + } + this.timeBetweenAttempsUnit = timeBetweenAttempsUnit; + } + + public int getTimeBetweenAttempts() { + return timeBetweenAttempts; + } + + public void setTimeBetweenAttempts(int timeBetweenAttempts) { + if(timeBetweenAttempts <= 0) { + throw new IllegalArgumentException("Time between attempts must be positive: " + numAttempts); + } + this.timeBetweenAttempts = timeBetweenAttempts; + } + + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java new file mode 100644 index 0000000000..e9e7d5bc60 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcaster.java @@ -0,0 +1,182 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import java.util.*; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastServicesBroadcaster; +import org.apache.nifi.io.socket.multicast.MulticastUtils; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Broadcasts services used by the clustering software using multicast communication. + * A configurable delay occurs after broadcasting the collection of services. + * + * The client caller is responsible for starting and stopping the broadcasting. + * The instance must be stopped before termination of the JVM to ensure proper + * resource clean-up. + * + * @author unattributed + */ +public class ClusterServicesBroadcaster implements MulticastServicesBroadcaster { + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(ClusterServicesBroadcaster.class)); + + private final Set services = new CopyOnWriteArraySet<>(); + + private final InetSocketAddress multicastAddress; + + private final MulticastConfiguration multicastConfiguration; + + private final ProtocolContext protocolContext; + + private final int broadcastDelayMs; + + private Timer broadcaster; + + private MulticastSocket multicastSocket; + + public ClusterServicesBroadcaster(final InetSocketAddress multicastAddress, + final MulticastConfiguration multicastConfiguration, + final ProtocolContext protocolContext, final String broadcastDelay) { + + if(multicastAddress == null) { + throw new IllegalArgumentException("Multicast address may not be null."); + } else if(multicastAddress.getAddress().isMulticastAddress() == false) { + throw new IllegalArgumentException("Multicast group address is not a Class D IP address."); + } else if(protocolContext == null) { + throw new IllegalArgumentException("Protocol Context may not be null."); + } else if(multicastConfiguration == null) { + throw new IllegalArgumentException("Multicast configuration may not be null."); + } + + this.services.addAll(services); + this.multicastAddress = multicastAddress; + this.multicastConfiguration = multicastConfiguration; + this.protocolContext = protocolContext; + this.broadcastDelayMs = (int) FormatUtils.getTimeDuration(broadcastDelay, TimeUnit.MILLISECONDS); + } + + public void start() throws IOException { + + if(isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + + // setup socket + multicastSocket = MulticastUtils.createMulticastSocket(multicastConfiguration); + + // setup broadcaster + broadcaster = new Timer("Cluster Services Broadcaster", /* is daemon */ true); + broadcaster.schedule(new TimerTask() { + @Override + public void run() { + for(final DiscoverableService service : services) { + try { + + final InetSocketAddress serviceAddress = service.getServiceAddress(); + logger.debug(String.format("Broadcasting Cluster Service '%s' at address %s:%d", + service.getServiceName(), serviceAddress.getHostName(), serviceAddress.getPort())); + + // create message + final ServiceBroadcastMessage msg = new ServiceBroadcastMessage(); + msg.setServiceName(service.getServiceName()); + msg.setAddress(serviceAddress.getHostName()); + msg.setPort(serviceAddress.getPort()); + + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + marshaller.marshal(msg, baos); + final byte[] packetBytes = baos.toByteArray(); + + // send message + final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, multicastAddress); + multicastSocket.send(packet); + + } catch(final Exception ex) { + logger.warn(String.format("Cluster Services Broadcaster failed broadcasting service '%s' due to: %s", service.getServiceName(), ex), ex); + } + } + } + }, 0, broadcastDelayMs); + } + + public boolean isRunning() { + return (broadcaster != null); + } + + public void stop() { + + if(isRunning() == false) { + throw new IllegalStateException("Instance is already stopped."); + } + + broadcaster.cancel(); + broadcaster = null; + + // close socket + MulticastUtils.closeQuietly(multicastSocket); + + } + + @Override + public int getBroadcastDelayMs() { + return broadcastDelayMs; + } + + @Override + public Set getServices() { + return Collections.unmodifiableSet(services); + } + + @Override + public InetSocketAddress getMulticastAddress() { + return multicastAddress; + } + + @Override + public boolean addService(final DiscoverableService service) { + return services.add(service); + } + + @Override + public boolean removeService(final String serviceName) { + for(final DiscoverableService service : services) { + if(service.getServiceName().equals(serviceName)) { + return services.remove(service); + } + } + return false; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java new file mode 100644 index 0000000000..680df65aec --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/CopyingInputStream.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.impl; + +import java.io.ByteArrayOutputStream; +import java.io.FilterInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +public class CopyingInputStream extends FilterInputStream { + private final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + private final int maxBytesToCopy; + private final InputStream in; + + public CopyingInputStream(final InputStream in, final int maxBytesToCopy) { + super(in); + this.maxBytesToCopy = maxBytesToCopy; + this.in = in; + } + + @Override + public int read() throws IOException { + final int delegateRead = in.read(); + if ( delegateRead != -1 && getNumberOfBytesCopied() < maxBytesToCopy ) { + baos.write(delegateRead); + } + + return delegateRead; + } + + @Override + public int read(byte[] b) throws IOException { + final int delegateRead = in.read(b); + if ( delegateRead >= 0 ) { + baos.write(b, 0, Math.min(delegateRead, maxBytesToCopy - getNumberOfBytesCopied())); + } + + return delegateRead; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + final int delegateRead = in.read(b, off, len); + if ( delegateRead >= 0 ) { + baos.write(b, off, Math.min(delegateRead, maxBytesToCopy - getNumberOfBytesCopied())); + } + + return delegateRead; + } + + public byte[] getBytesRead() { + return baos.toByteArray(); + } + + public void writeBytes(final OutputStream out) throws IOException { + baos.writeTo(out); + } + + public int getNumberOfBytesCopied() { + return baos.size(); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java new file mode 100644 index 0000000000..d3764b3d58 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListener.java @@ -0,0 +1,204 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import java.util.Collection; +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.CopyOnWriteArrayList; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastListener; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.reporting.Bulletin; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolListener; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.events.BulletinFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements a listener for protocol messages sent over multicast. If a message + * is of type MulticastProtocolMessage, then the underlying protocol message is + * passed to the handler. If the receiving handler produces a message response, + * then the message is wrapped with a MulticastProtocolMessage before being + * sent to the originator. + * + * The client caller is responsible for starting and stopping the listener. + * The instance must be stopped before termination of the JVM to ensure proper + * resource clean-up. + * + * @author unattributed + */ +public class MulticastProtocolListener extends MulticastListener implements ProtocolListener { + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(MulticastProtocolListener.class)); + + // immutable members + private final Collection handlers = new CopyOnWriteArrayList<>(); + private final String listenerId = UUID.randomUUID().toString(); + private final ProtocolContext protocolContext; + private volatile BulletinRepository bulletinRepository; + + public MulticastProtocolListener( + final int numThreads, + final InetSocketAddress multicastAddress, + final MulticastConfiguration configuration, + final ProtocolContext protocolContext) { + + super(numThreads, multicastAddress, configuration); + + if (protocolContext == null) { + throw new IllegalArgumentException("Protocol Context may not be null."); + } + this.protocolContext = protocolContext; + } + + @Override + public void setBulletinRepository(final BulletinRepository bulletinRepository) { + this.bulletinRepository = bulletinRepository; + } + + @Override + public void start() throws IOException { + + if(super.isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + + super.start(); + + } + + @Override + public void stop() throws IOException { + + if(super.isRunning() == false) { + throw new IllegalStateException("Instance is already stopped."); + } + + // shutdown listener + super.stop(); + + } + + @Override + public Collection getHandlers() { + return Collections.unmodifiableCollection(handlers); + } + + @Override + public void addHandler(final ProtocolHandler handler) { + if(handler == null) { + throw new NullPointerException("Protocol handler may not be null."); + } + handlers.add(handler); + } + + @Override + public boolean removeHandler(final ProtocolHandler handler) { + return handlers.remove(handler); + } + + @Override + public void dispatchRequest(final MulticastSocket multicastSocket, final DatagramPacket packet) { + + try { + + // unmarshall message + final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); + final ProtocolMessage request = unmarshaller.unmarshal(new ByteArrayInputStream(packet.getData(), 0, packet.getLength())); + + // unwrap multicast message, if necessary + final ProtocolMessage unwrappedRequest; + if(request instanceof MulticastProtocolMessage) { + final MulticastProtocolMessage multicastRequest = (MulticastProtocolMessage) request; + // don't process a message we sent + if(listenerId.equals(multicastRequest.getId())) { + return; + } else { + unwrappedRequest = multicastRequest.getProtocolMessage(); + } + } else { + unwrappedRequest = request; + } + + // dispatch message to handler + ProtocolHandler desiredHandler = null; + for (final ProtocolHandler handler : getHandlers()) { + if (handler.canHandle(unwrappedRequest)) { + desiredHandler = handler; + break; + } + } + + // if no handler found, throw exception; otherwise handle request + if (desiredHandler == null) { + throw new ProtocolException("No handler assigned to handle message type: " + request.getType()); + } else { + final ProtocolMessage response = desiredHandler.handle(request); + if(response != null) { + try { + + // wrap with listener id + final MulticastProtocolMessage multicastResponse = new MulticastProtocolMessage(listenerId, response); + + // marshal message + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(multicastResponse, baos); + final byte[] responseBytes = baos.toByteArray(); + + final int maxPacketSizeBytes = getMaxPacketSizeBytes(); + if(responseBytes.length > maxPacketSizeBytes) { + logger.warn("Cluster protocol handler '" + desiredHandler.getClass() + + "' produced a multicast response with length greater than configured max packet size '" + maxPacketSizeBytes + "'"); + } + + // create and send packet + final DatagramPacket responseDatagram = new DatagramPacket(responseBytes, responseBytes.length, getMulticastAddress().getAddress(), getMulticastAddress().getPort()); + multicastSocket.send(responseDatagram); + + } catch (final IOException ioe) { + throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to: " + ioe, ioe); + } + } + } + + } catch (final Throwable t) { + logger.warn("Failed processing protocol message due to " + t, t); + + if ( bulletinRepository != null ) { + final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", "Failed to process Protocol Message due to " + t.toString()); + bulletinRepository.addBulletin(bulletin); + } + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java new file mode 100644 index 0000000000..dc86d24ea2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImpl.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.impl; + +import java.io.IOException; +import java.net.Socket; + +import javax.net.ssl.SSLSocket; +import javax.security.cert.X509Certificate; + +import org.apache.nifi.cluster.protocol.NodeProtocolSender; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; +import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.io.socket.SocketUtils; +import org.apache.nifi.io.socket.multicast.DiscoverableService; + +public class NodeProtocolSenderImpl implements NodeProtocolSender { + private final SocketConfiguration socketConfiguration; + private final ClusterServiceLocator clusterManagerProtocolServiceLocator; + private final ProtocolContext protocolContext; + + public NodeProtocolSenderImpl(final ClusterServiceLocator clusterManagerProtocolServiceLocator, + final SocketConfiguration socketConfiguration, final ProtocolContext protocolContext) { + if(clusterManagerProtocolServiceLocator == null) { + throw new IllegalArgumentException("Protocol Service Locator may not be null."); + } else if(socketConfiguration == null) { + throw new IllegalArgumentException("Socket configuration may not be null."); + } else if(protocolContext == null) { + throw new IllegalArgumentException("Protocol Context may not be null."); + } + + this.clusterManagerProtocolServiceLocator = clusterManagerProtocolServiceLocator; + this.socketConfiguration = socketConfiguration; + this.protocolContext = protocolContext; + } + + + @Override + public ConnectionResponseMessage requestConnection(final ConnectionRequestMessage msg) throws ProtocolException, UnknownServiceAddressException { + Socket socket = null; + try { + socket = createSocket(); + + String ncmDn = null; + if ( socket instanceof SSLSocket ) { + final SSLSocket sslSocket = (SSLSocket) socket; + try { + final X509Certificate[] certChains = sslSocket.getSession().getPeerCertificateChain(); + if ( certChains != null && certChains.length > 0 ) { + ncmDn = certChains[0].getSubjectDN().getName(); + } + } catch (final ProtocolException pe) { + throw pe; + } catch (final Exception e) { + throw new ProtocolException(e); + } + } + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + + final ProtocolMessage response; + try { + // unmarshall response and return + final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); + response = unmarshaller.unmarshal(socket.getInputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed unmarshalling '" + MessageType.CONNECTION_RESPONSE + "' protocol message due to: " + ioe, ioe); + } + + if(MessageType.CONNECTION_RESPONSE == response.getType()) { + final ConnectionResponseMessage connectionResponse = (ConnectionResponseMessage) response; + connectionResponse.setClusterManagerDN(ncmDn); + return connectionResponse; + } else { + throw new ProtocolException("Expected message type '" + MessageType.CONNECTION_RESPONSE + "' but found '" + response.getType() + "'"); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + + + @Override + public void heartbeat(final HeartbeatMessage msg) throws ProtocolException, UnknownServiceAddressException { + sendProtocolMessage(msg); + } + + @Override + public void sendBulletins(NodeBulletinsMessage msg) throws ProtocolException, UnknownServiceAddressException { + sendProtocolMessage(msg); + } + + @Override + public void notifyControllerStartupFailure(final ControllerStartupFailureMessage msg) throws ProtocolException, UnknownServiceAddressException { + sendProtocolMessage(msg); + } + + @Override + public void notifyReconnectionFailure(ReconnectionFailureMessage msg) throws ProtocolException, UnknownServiceAddressException { + sendProtocolMessage(msg); + } + + private Socket createSocket() { + // determine the cluster manager's address + final DiscoverableService service = clusterManagerProtocolServiceLocator.getService(); + if(service == null) { + throw new UnknownServiceAddressException("Cluster Manager's service is not known. Verify a cluster manager is running."); + } + + try { + // create a socket + return SocketUtils.createSocket(service.getServiceAddress(), socketConfiguration); + } catch(final IOException ioe) { + throw new ProtocolException("Failed to create socket due to: " + ioe, ioe); + } + } + + private void sendProtocolMessage(final ProtocolMessage msg) { + Socket socket = null; + try { + socket = createSocket(); + + try { + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(msg, socket.getOutputStream()); + } catch(final IOException ioe) { + throw new ProtocolException("Failed marshalling '" + msg.getType() + "' protocol message due to: " + ioe, ioe); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + + public SocketConfiguration getSocketConfiguration() { + return socketConfiguration; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java new file mode 100644 index 0000000000..4b359f462f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderListener.java @@ -0,0 +1,115 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.util.Collection; + +import org.apache.nifi.cluster.protocol.NodeProtocolSender; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolListener; +import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; +import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage; +import org.apache.nifi.reporting.BulletinRepository; + +public class NodeProtocolSenderListener implements NodeProtocolSender, ProtocolListener { + + private final NodeProtocolSender sender; + private final ProtocolListener listener; + + public NodeProtocolSenderListener(final NodeProtocolSender sender, final ProtocolListener listener) { + if(sender == null) { + throw new IllegalArgumentException("NodeProtocolSender may not be null."); + } else if(listener == null) { + throw new IllegalArgumentException("ProtocolListener may not be null."); + } + this.sender = sender; + this.listener = listener; + } + + @Override + public void stop() throws IOException { + if(!isRunning()) { + throw new IllegalStateException("Instance is already stopped."); + } + listener.stop(); + } + + @Override + public void start() throws IOException { + if(isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + listener.start(); + } + + @Override + public boolean isRunning() { + return listener.isRunning(); + } + + @Override + public boolean removeHandler(final ProtocolHandler handler) { + return listener.removeHandler(handler); + } + + @Override + public Collection getHandlers() { + return listener.getHandlers(); + } + + @Override + public void addHandler(final ProtocolHandler handler) { + listener.addHandler(handler); + } + + @Override + public void heartbeat(final HeartbeatMessage msg) throws ProtocolException, UnknownServiceAddressException { + sender.heartbeat(msg); + } + + @Override + public ConnectionResponseMessage requestConnection(final ConnectionRequestMessage msg) throws ProtocolException, UnknownServiceAddressException { + return sender.requestConnection(msg); + } + + @Override + public void notifyControllerStartupFailure(final ControllerStartupFailureMessage msg) throws ProtocolException, UnknownServiceAddressException { + sender.notifyControllerStartupFailure(msg); + } + + @Override + public void notifyReconnectionFailure(final ReconnectionFailureMessage msg) throws ProtocolException, UnknownServiceAddressException { + sender.notifyReconnectionFailure(msg); + } + + @Override + public void sendBulletins(NodeBulletinsMessage msg) throws ProtocolException, UnknownServiceAddressException { + sender.sendBulletins(msg); + } + + @Override + public void setBulletinRepository(final BulletinRepository bulletinRepository) { + listener.setBulletinRepository(bulletinRepository); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java new file mode 100644 index 0000000000..ca30d9b2a5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListener.java @@ -0,0 +1,205 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.io.InputStream; +import java.net.Socket; +import java.util.Collection; +import java.util.Collections; +import java.util.UUID; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLSocket; +import javax.security.cert.X509Certificate; + +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolListener; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.events.BulletinFactory; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketListener; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.reporting.Bulletin; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.util.StopWatch; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements a listener for protocol messages sent over unicast socket. + * + * @author unattributed + */ +public class SocketProtocolListener extends SocketListener implements ProtocolListener { + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(SocketProtocolListener.class)); + private final ProtocolContext protocolContext; + private final Collection handlers = new CopyOnWriteArrayList<>(); + private volatile BulletinRepository bulletinRepository; + + public SocketProtocolListener( + final int numThreads, + final int port, + final ServerSocketConfiguration configuration, + final ProtocolContext protocolContext) { + + super(numThreads, port, configuration); + + if(protocolContext == null) { + throw new IllegalArgumentException("Protocol Context may not be null."); + } + + this.protocolContext = protocolContext; + } + + @Override + public void setBulletinRepository(final BulletinRepository bulletinRepository) { + this.bulletinRepository = bulletinRepository; + } + + @Override + public void start() throws IOException { + + if(super.isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + + super.start(); + } + + @Override + public void stop() throws IOException { + + if(super.isRunning() == false) { + throw new IOException("Instance is already stopped."); + } + + super.stop(); + + } + + @Override + public Collection getHandlers() { + return Collections.unmodifiableCollection(handlers); + } + + @Override + public void addHandler(final ProtocolHandler handler) { + if(handler == null) { + throw new NullPointerException("Protocol handler may not be null."); + } + handlers.add(handler); + } + + @Override + public boolean removeHandler(final ProtocolHandler handler) { + return handlers.remove(handler); + } + + @Override + public void dispatchRequest(final Socket socket) { + byte[] receivedMessage = null; + String hostname = null; + final int maxMsgBuffer = 1024 * 1024; // don't buffer more than 1 MB of the message + try { + final StopWatch stopWatch = new StopWatch(true); + hostname = socket.getInetAddress().getHostName(); + final String requestId = UUID.randomUUID().toString(); + logger.info("Received request {} from {}", requestId, hostname); + + String requestorDn = null; + if ( socket instanceof SSLSocket ) { + final SSLSocket sslSocket = (SSLSocket) socket; + try { + final X509Certificate[] certChains = sslSocket.getSession().getPeerCertificateChain(); + if ( certChains != null && certChains.length > 0 ) { + requestorDn = certChains[0].getSubjectDN().getName(); + } + } catch (final ProtocolException pe) { + throw pe; + } catch (final Exception e) { + throw new ProtocolException(e); + } + } + + // unmarshall message + final ProtocolMessageUnmarshaller unmarshaller = protocolContext.createUnmarshaller(); + final InputStream inStream = socket.getInputStream(); + final CopyingInputStream copyingInputStream = new CopyingInputStream(inStream, maxMsgBuffer); // don't copy more than 1 MB + logger.debug("Request {} has a message length of {}", requestId, copyingInputStream.getNumberOfBytesCopied()); + + final ProtocolMessage request; + try { + request = unmarshaller.unmarshal(copyingInputStream); + } finally { + receivedMessage = copyingInputStream.getBytesRead(); + } + + request.setRequestorDN(requestorDn); + + // dispatch message to handler + ProtocolHandler desiredHandler = null; + for (final ProtocolHandler handler : getHandlers()) { + if (handler.canHandle(request)) { + desiredHandler = handler; + break; + } + } + + // if no handler found, throw exception; otherwise handle request + if (desiredHandler == null) { + throw new ProtocolException("No handler assigned to handle message type: " + request.getType()); + } else { + final ProtocolMessage response = desiredHandler.handle(request); + if(response != null) { + try { + logger.debug("Sending response for request {}", requestId); + + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + marshaller.marshal(response, socket.getOutputStream()); + } catch (final IOException ioe) { + throw new ProtocolException("Failed marshalling protocol message in response to message type: " + request.getType() + " due to " + ioe, ioe); + } + } + } + + stopWatch.stop(); + logger.info("Finished processing request {} (type={}, length={} bytes) in {} millis", requestId, request.getType(), receivedMessage.length, stopWatch.getDuration(TimeUnit.MILLISECONDS)); + } catch (final IOException e) { + logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e); + + if ( bulletinRepository != null ) { + final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", String.format("Failed to process protocol message from %s due to: %s", hostname, e.toString())); + bulletinRepository.addBulletin(bulletin); + } + } catch (final ProtocolException e) { + logger.warn("Failed processing protocol message from " + hostname + " due to " + e, e); + if ( bulletinRepository != null ) { + final Bulletin bulletin = BulletinFactory.createBulletin("Clustering", "WARNING", String.format("Failed to process protocol message from %s due to: %s", hostname, e.toString())); + bulletinRepository.addBulletin(bulletin); + } + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java new file mode 100644 index 0000000000..bc686306c0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/JaxbProtocolContext.java @@ -0,0 +1,148 @@ +/* + * 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.cluster.protocol.jaxb; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; +import javax.xml.bind.Unmarshaller; + +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; + +/** + * Implements a context for communicating internally amongst the cluster using + * JAXB. + * + * @param The type of protocol message. + * + * @author unattributed + */ +public class JaxbProtocolContext implements ProtocolContext { + + private static final int BUF_SIZE = (int) Math.pow(2, 10); // 1k + + /* + * A sentinel is used to detect corrupted messages. Relying on the integrity + * of the message size can cause memory issues if the value is corrupted + * and equal to a number larger than the memory size. + */ + private static final byte MESSAGE_PROTOCOL_START_SENTINEL = 0x5A; + + private final JAXBContext jaxbCtx; + + public JaxbProtocolContext(final JAXBContext jaxbCtx) { + this.jaxbCtx = jaxbCtx; + } + + @Override + public ProtocolMessageMarshaller createMarshaller() { + return new ProtocolMessageMarshaller() { + + @Override + public void marshal(final T msg, final OutputStream os) throws IOException { + + try { + + // marshal message to output stream + final Marshaller marshaller = jaxbCtx.createMarshaller(); + final ByteArrayOutputStream msgBytes = new ByteArrayOutputStream(); + marshaller.marshal(msg, msgBytes); + + final DataOutputStream dos = new DataOutputStream(os); + + // write message protocol sentinel + dos.write(MESSAGE_PROTOCOL_START_SENTINEL); + + // write message size in bytes + dos.writeInt(msgBytes.size()); + + // write message + dos.write(msgBytes.toByteArray()); + + dos.flush(); + + } catch (final JAXBException je) { + throw new IOException("Failed marshalling protocol message due to: " + je, je); + } + + } + }; + } + + @Override + public ProtocolMessageUnmarshaller createUnmarshaller() { + return new ProtocolMessageUnmarshaller() { + + @Override + public T unmarshal(final InputStream is) throws IOException { + + try { + + final DataInputStream dis = new DataInputStream(is); + + // check for the presence of the message protocol sentinel + final byte sentinel = (byte) dis.read(); + if ( sentinel == -1 ) { + throw new EOFException(); + } + + if(MESSAGE_PROTOCOL_START_SENTINEL != sentinel) { + throw new IOException("Failed reading protocol message due to malformed header"); + } + + // read the message size + final int msgBytesSize = dis.readInt(); + + // read the message + final ByteBuffer buffer = ByteBuffer.allocate(msgBytesSize); + int totalBytesRead = 0; + do { + final int bytesToRead; + if ((msgBytesSize - totalBytesRead) >= BUF_SIZE) { + bytesToRead = BUF_SIZE; + } else { + bytesToRead = msgBytesSize - totalBytesRead; + } + totalBytesRead += dis.read(buffer.array(), totalBytesRead, bytesToRead); + } while (totalBytesRead < msgBytesSize); + + // unmarshall message and return + final Unmarshaller unmarshaller = jaxbCtx.createUnmarshaller(); + final byte[] msg = new byte[totalBytesRead]; + buffer.get(msg); + return (T) unmarshaller.unmarshal(new ByteArrayInputStream(msg)); + + } catch (final JAXBException je) { + throw new IOException("Failed unmarshalling protocol message due to: " + je, je); + } + + } + }; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java new file mode 100644 index 0000000000..d9de24f0b9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionRequest.java @@ -0,0 +1,40 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * @author unattributed + */ +public class AdaptedConnectionRequest { + + private NodeIdentifier nodeIdentifier; + + public AdaptedConnectionRequest() {} + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public void setNodeIdentifier(final NodeIdentifier nodeIdentifier) { + this.nodeIdentifier = nodeIdentifier; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java new file mode 100644 index 0000000000..c7c783bdc2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedConnectionResponse.java @@ -0,0 +1,109 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; + +/** + * @author unattributed + */ +public class AdaptedConnectionResponse { + + private StandardDataFlow dataFlow; + private NodeIdentifier nodeIdentifier; + private boolean blockedByFirewall; + private boolean primary; + private int tryLaterSeconds; + private Integer managerRemoteInputPort; + private Boolean managerRemoteCommsSecure; + private String instanceId; + + public AdaptedConnectionResponse() {} + + @XmlJavaTypeAdapter(DataFlowAdapter.class) + public StandardDataFlow getDataFlow() { + return dataFlow; + } + + public void setDataFlow(StandardDataFlow dataFlow) { + this.dataFlow = dataFlow; + } + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public void setNodeIdentifier(NodeIdentifier nodeIdentifier) { + this.nodeIdentifier = nodeIdentifier; + } + + public int getTryLaterSeconds() { + return tryLaterSeconds; + } + + public void setTryLaterSeconds(int tryLaterSeconds) { + this.tryLaterSeconds = tryLaterSeconds; + } + + public boolean isBlockedByFirewall() { + return blockedByFirewall; + } + + public void setBlockedByFirewall(boolean blockedByFirewall) { + this.blockedByFirewall = blockedByFirewall; + } + + public boolean isPrimary() { + return primary; + } + + public void setPrimary(boolean primary) { + this.primary = primary; + } + + public boolean shouldTryLater() { + return tryLaterSeconds > 0; + } + + public void setManagerRemoteInputPort(Integer managerRemoteInputPort) { + this.managerRemoteInputPort = managerRemoteInputPort; + } + + public Integer getManagerRemoteInputPort() { + return managerRemoteInputPort; + } + + public void setManagerRemoteCommsSecure(Boolean secure) { + this.managerRemoteCommsSecure = secure; + } + + public Boolean isManagerRemoteCommsSecure() { + return managerRemoteCommsSecure; + } + + public void setInstanceId(String instanceId) { + this.instanceId = instanceId; + } + + public String getInstanceId() { + return instanceId; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java new file mode 100644 index 0000000000..89d903bd6a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedCounter.java @@ -0,0 +1,56 @@ +/* + * 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.cluster.protocol.jaxb.message; + +/** + * @author unattributed + */ +public class AdaptedCounter { + + private String groupName; + + private String name; + + private long value; + + public AdaptedCounter() {} + + public String getGroupName() { + return groupName; + } + + public void setGroupName(String counterGroupName) { + this.groupName = counterGroupName; + } + + public String getName() { + return name; + } + + public void setName(String counterName) { + this.name = counterName; + } + + public long getValue() { + return value; + } + + public void setValue(long value) { + this.value = value; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java new file mode 100644 index 0000000000..bb976195c5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedDataFlow.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.jaxb.message; + +/** + * @author unattributed + */ +public class AdaptedDataFlow { + + private byte[] flow; + private byte[] templates; + private byte[] snippets; + + private boolean autoStartProcessors; + + public AdaptedDataFlow() {} + + public byte[] getFlow() { + return flow; + } + + public void setFlow(byte[] flow) { + this.flow = flow; + } + + public byte[] getTemplates() { + return templates; + } + + public void setTemplates(byte[] templates) { + this.templates = templates; + } + + public byte[] getSnippets() { + return snippets; + } + + public void setSnippets(byte[] snippets) { + this.snippets = snippets; + } + + public boolean isAutoStartProcessors() { + return autoStartProcessors; + } + + public void setAutoStartProcessors(boolean runningAllProcessors) { + this.autoStartProcessors = runningAllProcessors; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java new file mode 100644 index 0000000000..5b9d9b7619 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedHeartbeat.java @@ -0,0 +1,66 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * @author unattributed + */ +public class AdaptedHeartbeat { + + private NodeIdentifier nodeIdentifier; + private byte[] payload; + private boolean primary; + private boolean connected; + + public AdaptedHeartbeat() {} + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public void setNodeIdentifier(NodeIdentifier nodeIdentifier) { + this.nodeIdentifier = nodeIdentifier; + } + + public boolean isPrimary() { + return primary; + } + + public void setPrimary(boolean primary) { + this.primary = primary; + } + + public boolean isConnected() { + return connected; + } + + public void setConnected(boolean connected) { + this.connected = connected; + } + + public byte[] getPayload() { + return payload; + } + + public void setPayload(byte[] payload) { + this.payload = payload; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java new file mode 100644 index 0000000000..98e2438a97 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeBulletins.java @@ -0,0 +1,50 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * @author unattributed + */ +public class AdaptedNodeBulletins { + + private NodeIdentifier nodeIdentifier; + + private byte[] payload; + + public AdaptedNodeBulletins() {} + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeIdentifier() { + return nodeIdentifier; + } + + public void setNodeIdentifier(NodeIdentifier nodeIdentifier) { + this.nodeIdentifier = nodeIdentifier; + } + + public byte[] getPayload() { + return payload; + } + + public void setPayload(byte[] payload) { + this.payload = payload; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java new file mode 100644 index 0000000000..8134ea302a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/AdaptedNodeIdentifier.java @@ -0,0 +1,76 @@ +/* + * 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.cluster.protocol.jaxb.message; + +/** + * @author unattributed + */ +public class AdaptedNodeIdentifier { + + private String id; + + private String apiAddress; + + private int apiPort; + + private String socketAddress; + + private int socketPort; + + public AdaptedNodeIdentifier() {} + + public String getApiAddress() { + return apiAddress; + } + + public void setApiAddress(String apiAddress) { + this.apiAddress = apiAddress; + } + + public int getApiPort() { + return apiPort; + } + + public void setApiPort(int apiPort) { + this.apiPort = apiPort; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public String getSocketAddress() { + return socketAddress; + } + + public void setSocketAddress(String socketAddress) { + this.socketAddress = socketAddress; + } + + public int getSocketPort() { + return socketPort; + } + + public void setSocketPort(int socketPort) { + this.socketPort = socketPort; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java new file mode 100644 index 0000000000..1f91cf1461 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionRequestAdapter.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlAdapter; +import org.apache.nifi.cluster.protocol.ConnectionRequest; + +/** + * @author unattributed + */ +public class ConnectionRequestAdapter extends XmlAdapter { + + @Override + public AdaptedConnectionRequest marshal(final ConnectionRequest cr) { + final AdaptedConnectionRequest aCr = new AdaptedConnectionRequest(); + if(cr != null) { + aCr.setNodeIdentifier(cr.getProposedNodeIdentifier()); + } + return aCr; + } + + @Override + public ConnectionRequest unmarshal(final AdaptedConnectionRequest aCr) { + return new ConnectionRequest(aCr.getNodeIdentifier()); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java new file mode 100644 index 0000000000..143bab0eef --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ConnectionResponseAdapter.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlAdapter; +import org.apache.nifi.cluster.protocol.ConnectionResponse; + +/** + * @author unattributed + */ +public class ConnectionResponseAdapter extends XmlAdapter { + + @Override + public AdaptedConnectionResponse marshal(final ConnectionResponse cr) { + final AdaptedConnectionResponse aCr = new AdaptedConnectionResponse(); + if(cr != null) { + aCr.setDataFlow(cr.getDataFlow()); + aCr.setNodeIdentifier(cr.getNodeIdentifier()); + aCr.setTryLaterSeconds(cr.getTryLaterSeconds()); + aCr.setBlockedByFirewall(cr.isBlockedByFirewall()); + aCr.setPrimary(cr.isPrimary()); + aCr.setManagerRemoteInputPort(cr.getManagerRemoteInputPort()); + aCr.setManagerRemoteCommsSecure(cr.isManagerRemoteCommsSecure()); + aCr.setInstanceId(cr.getInstanceId()); + } + return aCr; + } + + @Override + public ConnectionResponse unmarshal(final AdaptedConnectionResponse aCr) { + if(aCr.shouldTryLater()) { + return new ConnectionResponse(aCr.getTryLaterSeconds()); + } else if(aCr.isBlockedByFirewall()) { + return ConnectionResponse.createBlockedByFirewallResponse(); + } else { + return new ConnectionResponse(aCr.getNodeIdentifier(), aCr.getDataFlow(), aCr.isPrimary(), + aCr.getManagerRemoteInputPort(), aCr.isManagerRemoteCommsSecure(), aCr.getInstanceId()); + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java new file mode 100644 index 0000000000..8d9467fbcb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/DataFlowAdapter.java @@ -0,0 +1,50 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlAdapter; + +import org.apache.nifi.cluster.protocol.StandardDataFlow; + +/** + * @author unattributed + */ +public class DataFlowAdapter extends XmlAdapter { + + @Override + public AdaptedDataFlow marshal(final StandardDataFlow df) { + + final AdaptedDataFlow aDf = new AdaptedDataFlow(); + + if(df != null) { + aDf.setFlow(df.getFlow()); + aDf.setTemplates(df.getTemplates()); + aDf.setSnippets(df.getSnippets()); + aDf.setAutoStartProcessors(df.isAutoStartProcessors()); + } + + return aDf; + } + + @Override + public StandardDataFlow unmarshal(final AdaptedDataFlow aDf) { + final StandardDataFlow dataFlow = new StandardDataFlow(aDf.getFlow(), aDf.getTemplates(), aDf.getSnippets()); + dataFlow.setAutoStartProcessors(aDf.isAutoStartProcessors()); + return dataFlow; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java new file mode 100644 index 0000000000..0e073b6dec --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/HeartbeatAdapter.java @@ -0,0 +1,54 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlAdapter; +import org.apache.nifi.cluster.protocol.Heartbeat; + +/** + * @author unattributed + */ +public class HeartbeatAdapter extends XmlAdapter { + + @Override + public AdaptedHeartbeat marshal(final Heartbeat hb) { + + final AdaptedHeartbeat aHb = new AdaptedHeartbeat(); + + if(hb != null) { + // set node identifier + aHb.setNodeIdentifier(hb.getNodeIdentifier()); + + // set payload + aHb.setPayload(hb.getPayload()); + + // set leader flag + aHb.setPrimary(hb.isPrimary()); + + // set connected flag + aHb.setConnected(hb.isConnected()); + } + + return aHb; + } + + @Override + public Heartbeat unmarshal(final AdaptedHeartbeat aHb) { + return new Heartbeat(aHb.getNodeIdentifier(), aHb.isPrimary(), aHb.isConnected(), aHb.getPayload()); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java new file mode 100644 index 0000000000..c3a57f54bc --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/JaxbProtocolUtils.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; + +/** + * @author unattributed + */ +public final class JaxbProtocolUtils { + + public static final String JAXB_CONTEXT_PATH = ObjectFactory.class.getPackage().getName(); + + public static final JAXBContext JAXB_CONTEXT = initializeJaxbContext(); + + /** + * Load the JAXBContext version. + */ + private static JAXBContext initializeJaxbContext() { + try { + return JAXBContext.newInstance(JAXB_CONTEXT_PATH); + } catch (JAXBException e) { + throw new RuntimeException("Unable to create JAXBContext.", e); + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java new file mode 100644 index 0000000000..1ae41f78a4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeBulletinsAdapter.java @@ -0,0 +1,48 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlAdapter; +import org.apache.nifi.cluster.protocol.NodeBulletins; + +/** + * @author unattributed + */ +public class NodeBulletinsAdapter extends XmlAdapter { + + @Override + public AdaptedNodeBulletins marshal(final NodeBulletins hb) { + + final AdaptedNodeBulletins adaptedBulletins = new AdaptedNodeBulletins(); + + if(hb != null) { + // set node identifier + adaptedBulletins.setNodeIdentifier(hb.getNodeIdentifier()); + + // set payload + adaptedBulletins.setPayload(hb.getPayload()); + } + + return adaptedBulletins; + } + + @Override + public NodeBulletins unmarshal(final AdaptedNodeBulletins adaptedBulletins) { + return new NodeBulletins(adaptedBulletins.getNodeIdentifier(), adaptedBulletins.getPayload()); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java new file mode 100644 index 0000000000..fe2d8a44e3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/NodeIdentifierAdapter.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.adapters.XmlAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * @author unattributed + */ +public class NodeIdentifierAdapter extends XmlAdapter { + + @Override + public AdaptedNodeIdentifier marshal(final NodeIdentifier ni) { + if(ni == null) { + return null; + } else { + final AdaptedNodeIdentifier aNi = new AdaptedNodeIdentifier(); + aNi.setId(ni.getId()); + aNi.setApiAddress(ni.getApiAddress()); + aNi.setApiPort(ni.getApiPort()); + aNi.setSocketAddress(ni.getSocketAddress()); + aNi.setSocketPort(ni.getSocketPort()); + return aNi; + } + } + + @Override + public NodeIdentifier unmarshal(final AdaptedNodeIdentifier aNi) { + if(aNi == null) { + return null; + } else { + return new NodeIdentifier(aNi.getId(), aNi.getApiAddress(), aNi.getApiPort(), aNi.getSocketAddress(), aNi.getSocketPort()); + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java new file mode 100644 index 0000000000..16135368ae --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/jaxb/message/ObjectFactory.java @@ -0,0 +1,104 @@ +/* + * 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.cluster.protocol.jaxb.message; + +import javax.xml.bind.annotation.XmlRegistry; + +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.DisconnectMessage; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; +import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ReconnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; + +/** + * @author unattributed + */ +@XmlRegistry +public class ObjectFactory { + + public ObjectFactory() {} + + public ReconnectionRequestMessage createReconnectionRequestMessage() { + return new ReconnectionRequestMessage(); + } + + public ReconnectionFailureMessage createReconnectionFailureMessage() { + return new ReconnectionFailureMessage(); + } + + public ReconnectionResponseMessage createReconnectionResponseMessage() { + return new ReconnectionResponseMessage(); + } + + public DisconnectMessage createDisconnectionMessage() { + return new DisconnectMessage(); + } + + public ConnectionRequestMessage createConnectionRequestMessage() { + return new ConnectionRequestMessage(); + } + + public ConnectionResponseMessage createConnectionResponseMessage() { + return new ConnectionResponseMessage(); + } + + public ServiceBroadcastMessage createServiceBroadcastMessage() { + return new ServiceBroadcastMessage(); + } + + public HeartbeatMessage createHeartbeatMessage() { + return new HeartbeatMessage(); + } + + public FlowRequestMessage createFlowRequestMessage() { + return new FlowRequestMessage(); + } + + public FlowResponseMessage createFlowResponseMessage() { + return new FlowResponseMessage(); + } + + public PingMessage createPingMessage() { + return new PingMessage(); + } + + public MulticastProtocolMessage createMulticastProtocolMessage() { + return new MulticastProtocolMessage(); + } + + public ControllerStartupFailureMessage createControllerStartupFailureMessage() { + return new ControllerStartupFailureMessage(); + } + + public PrimaryRoleAssignmentMessage createPrimaryRoleAssignmentMessage() { + return new PrimaryRoleAssignmentMessage(); + } + + public NodeBulletinsMessage createBulletinsMessage() { + return new NodeBulletinsMessage(); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.java new file mode 100644 index 0000000000..344de4ef21 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionRequestMessage.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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.nifi.cluster.protocol.ConnectionRequest; + +/** + * @author unattributed + */ +@XmlRootElement(name = "connectionRequestMessage") +public class ConnectionRequestMessage extends ProtocolMessage { + + private ConnectionRequest connectionRequest; + + public ConnectionRequestMessage() {} + + public ConnectionRequest getConnectionRequest() { + return connectionRequest; + } + + public void setConnectionRequest(ConnectionRequest connectionRequest) { + this.connectionRequest = connectionRequest; + } + + @Override + public MessageType getType() { + return MessageType.CONNECTION_REQUEST; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java new file mode 100644 index 0000000000..a262d7a98a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ConnectionResponseMessage.java @@ -0,0 +1,66 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import org.apache.nifi.cluster.protocol.ConnectionResponse; + +/** + * @author unattributed + */ +@XmlRootElement(name = "connectionResponseMessage") +public class ConnectionResponseMessage extends ProtocolMessage { + + private ConnectionResponse connectionResponse; + private String clusterManagerDN; + + public ConnectionResponseMessage() {} + + public ConnectionResponse getConnectionResponse() { + return connectionResponse; + } + + public void setConnectionResponse(final ConnectionResponse connectionResponse) { + this.connectionResponse = connectionResponse; + + if ( clusterManagerDN != null ) { + this.connectionResponse.setClusterManagerDN(clusterManagerDN); + } + } + + public void setClusterManagerDN(final String dn) { + if ( connectionResponse != null ) { + connectionResponse.setClusterManagerDN(dn); + } + this.clusterManagerDN = dn; + } + + /** + * Returns the DN of the NCM, if it is available or null otherwise. + * + * @return + */ + public String getClusterManagerDN() { + return clusterManagerDN; + } + + @Override + public MessageType getType() { + return MessageType.CONNECTION_RESPONSE; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java new file mode 100644 index 0000000000..ebc1caec0b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ControllerStartupFailureMessage.java @@ -0,0 +1,49 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +/** + * @author unattributed + */ +@XmlRootElement(name = "controllerStartupFailureMessage") +public class ControllerStartupFailureMessage extends ExceptionMessage { + + private NodeIdentifier nodeId; + + public ControllerStartupFailureMessage() {} + + @Override + public MessageType getType() { + return MessageType.CONTROLLER_STARTUP_FAILURE; + } + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java new file mode 100644 index 0000000000..8aa7a4008b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/DisconnectMessage.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +/** + * @author unattributed + */ +@XmlRootElement(name = "disconnectionMessage") +public class DisconnectMessage extends ProtocolMessage { + + private NodeIdentifier nodeId; + private String explanation; + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + + public String getExplanation() { + return explanation; + } + + public void setExplanation(String explanation) { + this.explanation = explanation; + } + + @Override + public MessageType getType() { + return MessageType.DISCONNECTION_REQUEST; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java new file mode 100644 index 0000000000..99a6deea9f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ExceptionMessage.java @@ -0,0 +1,44 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * @author unattributed + */ +@XmlRootElement(name = "exceptionMessage") +public class ExceptionMessage extends ProtocolMessage { + + private String exceptionMessage; + + public ExceptionMessage() {} + + public String getExceptionMessage() { + return exceptionMessage; + } + + public void setExceptionMessage(String exceptionMessage) { + this.exceptionMessage = exceptionMessage; + } + + @Override + public MessageType getType() { + return MessageType.EXCEPTION; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.java new file mode 100644 index 0000000000..4a10538bb1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowRequestMessage.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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +/** + * @author unattributed + */ +@XmlRootElement(name = "flowRequestMessage") +public class FlowRequestMessage extends ProtocolMessage { + + private NodeIdentifier nodeId; + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + + @Override + public MessageType getType() { + return MessageType.FLOW_REQUEST; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java new file mode 100644 index 0000000000..0d34dae7ca --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/FlowResponseMessage.java @@ -0,0 +1,44 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +import org.apache.nifi.cluster.protocol.StandardDataFlow; + +/** + * @author unattributed + */ +@XmlRootElement(name = "flowResponseMessage") +public class FlowResponseMessage extends ProtocolMessage { + + private StandardDataFlow dataFlow; + + @Override + public MessageType getType() { + return MessageType.FLOW_RESPONSE; + } + + public StandardDataFlow getDataFlow() { + return dataFlow; + } + + public void setDataFlow(StandardDataFlow dataFlow) { + this.dataFlow = dataFlow; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java new file mode 100644 index 0000000000..0064cb699c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/HeartbeatMessage.java @@ -0,0 +1,43 @@ +/* + * 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.cluster.protocol.message; + +import org.apache.nifi.cluster.protocol.Heartbeat; +import javax.xml.bind.annotation.XmlRootElement; + +/** + * @author unattributed + */ +@XmlRootElement(name = "heartbeatMessage") +public class HeartbeatMessage extends ProtocolMessage { + + private Heartbeat heartbeat; + + @Override + public MessageType getType() { + return MessageType.HEARTBEAT; + } + + public Heartbeat getHeartbeat() { + return heartbeat; + } + + public void setHeartbeat(Heartbeat heartbeat) { + this.heartbeat = heartbeat; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java new file mode 100644 index 0000000000..c6d2d44ed4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/MulticastProtocolMessage.java @@ -0,0 +1,66 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * Wraps a protocol message and an identifier for sending the message by way + * multicast. The identifier is necessary for the sender to identify a message + * sent by it. + * + * @author unattributed + */ +@XmlRootElement(name = "multicastMessage") +public class MulticastProtocolMessage extends ProtocolMessage { + + private ProtocolMessage protocolMessage; + + private String id; + + public MulticastProtocolMessage() {} + + public MulticastProtocolMessage(final String id, final ProtocolMessage protocolMessage) { + this.protocolMessage = protocolMessage; + this.id = id; + } + + @Override + public MessageType getType() { + if(protocolMessage == null) { + return null; + } + return protocolMessage.getType(); + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + public ProtocolMessage getProtocolMessage() { + return protocolMessage; + } + + public void setProtocolMessage(ProtocolMessage protocolMessage) { + this.protocolMessage = protocolMessage; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java new file mode 100644 index 0000000000..9237a924fa --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/NodeBulletinsMessage.java @@ -0,0 +1,43 @@ +/* + * 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.cluster.protocol.message; + +import org.apache.nifi.cluster.protocol.NodeBulletins; +import javax.xml.bind.annotation.XmlRootElement; + +/** + * @author unattributed + */ +@XmlRootElement(name = "nodeBulletinsMessage") +public class NodeBulletinsMessage extends ProtocolMessage { + + private NodeBulletins bulletins; + + @Override + public MessageType getType() { + return MessageType.BULLETINS; + } + + public NodeBulletins getBulletins() { + return bulletins; + } + + public void setBulletins(NodeBulletins bulletins) { + this.bulletins = bulletins; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java new file mode 100644 index 0000000000..ee38deba70 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PingMessage.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.message; + +import java.util.Date; +import javax.xml.bind.annotation.XmlRootElement; + +/** + * @author unattributed + */ +@XmlRootElement(name = "pingMessage") +public class PingMessage extends ProtocolMessage { + + private String id; + + private Date date = new Date(); + + public PingMessage() {} + + public Date getDate() { + return date; + } + + public void setDate(Date date) { + this.date = date; + } + + public String getId() { + return id; + } + + public void setId(String id) { + this.id = id; + } + + @Override + public MessageType getType() { + return MessageType.PING; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java new file mode 100644 index 0000000000..a289abc977 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/PrimaryRoleAssignmentMessage.java @@ -0,0 +1,56 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +/** + * @author unattributed + */ +@XmlRootElement(name = "primaryRoleAssignmentMessage") +public class PrimaryRoleAssignmentMessage extends ProtocolMessage { + + private NodeIdentifier nodeId; + + private boolean primary; + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + + public boolean isPrimary() { + return primary; + } + + public void setPrimary(boolean primary) { + this.primary = primary; + } + + @Override + public MessageType getType() { + return MessageType.PRIMARY_ROLE; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java new file mode 100644 index 0000000000..6bf2a13237 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ProtocolMessage.java @@ -0,0 +1,61 @@ +/* + * 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.cluster.protocol.message; + +/** + * @author unattributed + */ +public abstract class ProtocolMessage { + private volatile String requestorDN; + + public static enum MessageType { + BULLETINS, + CONNECTION_REQUEST, + CONNECTION_RESPONSE, + CONTROLLER_STARTUP_FAILURE, + RECONNECTION_FAILURE, + DISCONNECTION_REQUEST, + EXCEPTION, + FLOW_REQUEST, + FLOW_RESPONSE, + HEARTBEAT, + PING, + PRIMARY_ROLE, + RECONNECTION_REQUEST, + RECONNECTION_RESPONSE, + SERVICE_BROADCAST, + } + + public abstract MessageType getType(); + + /** + * Sets the DN of the entity making the request + * @param dn + */ + public void setRequestorDN(final String dn) { + this.requestorDN = dn; + } + + /** + * Returns the DN of the entity that made the request, if using a secure socket. Otherwise, returns null + * @return + */ + public String getRequestorDN() { + return requestorDN; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java new file mode 100644 index 0000000000..ba45e281b9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionFailureMessage.java @@ -0,0 +1,45 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +@XmlRootElement(name = "reconnectionFailureMessage") +public class ReconnectionFailureMessage extends ExceptionMessage { + private NodeIdentifier nodeId; + + public ReconnectionFailureMessage() {} + + @Override + public MessageType getType() { + return MessageType.RECONNECTION_FAILURE; + } + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java new file mode 100644 index 0000000000..eab3d5d588 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionRequestMessage.java @@ -0,0 +1,94 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; + +/** + * @author unattributed + */ +@XmlRootElement(name = "reconnectionRequestMessage") +public class ReconnectionRequestMessage extends ProtocolMessage { + + private NodeIdentifier nodeId; + private StandardDataFlow dataFlow; + private boolean primary; + private Integer managerRemoteSiteListeningPort; + private Boolean managerRemoteSiteCommsSecure; + private String instanceId; + + public ReconnectionRequestMessage() {} + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getNodeId() { + return nodeId; + } + + public void setNodeId(NodeIdentifier nodeId) { + this.nodeId = nodeId; + } + + public StandardDataFlow getDataFlow() { + return dataFlow; + } + + public void setDataFlow(StandardDataFlow dataFlow) { + this.dataFlow = dataFlow; + } + + public boolean isPrimary() { + return primary; + } + + public void setPrimary(boolean primary) { + this.primary = primary; + } + + @Override + public MessageType getType() { + return MessageType.RECONNECTION_REQUEST; + } + + public void setManagerRemoteSiteListeningPort(final Integer listeningPort) { + this.managerRemoteSiteListeningPort = listeningPort; + } + + public Integer getManagerRemoteSiteListeningPort() { + return managerRemoteSiteListeningPort; + } + + public void setManagerRemoteSiteCommsSecure(final Boolean remoteSiteCommsSecure) { + this.managerRemoteSiteCommsSecure = remoteSiteCommsSecure; + } + + public Boolean isManagerRemoteSiteCommsSecure() { + return managerRemoteSiteCommsSecure; + } + + public void setInstanceId(final String instanceId) { + this.instanceId = instanceId; + } + + public String getInstanceId() { + return instanceId; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java new file mode 100644 index 0000000000..fd0f921d6e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ReconnectionResponseMessage.java @@ -0,0 +1,32 @@ +/* + * 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.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * This message is used an "ACK" for a ReconnectionRequestMessage + */ +@XmlRootElement(name = "reconnectionResponseMessage") +public class ReconnectionResponseMessage extends ProtocolMessage { + + @Override + public MessageType getType() { + return MessageType.RECONNECTION_RESPONSE; + } + +} \ No newline at end of file diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java new file mode 100644 index 0000000000..92708ba68f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/message/ServiceBroadcastMessage.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.message; + +import javax.xml.bind.annotation.XmlRootElement; + +/** + * @author unattributed + */ +@XmlRootElement(name = "serviceBroadcastMessage") +public class ServiceBroadcastMessage extends ProtocolMessage { + + private String serviceName; + + private String address; + + private int port; + + public ServiceBroadcastMessage() {} + + public String getServiceName() { + return serviceName; + } + + public void setServiceName(String serviceName) { + this.serviceName = serviceName; + } + + public String getAddress() { + return address; + } + + public void setAddress(String address) { + this.address = address; + } + + public int getPort() { + return port; + } + + public void setPort(int port) { + this.port = port; + } + + @Override + public MessageType getType() { + return MessageType.SERVICE_BROADCAST; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java new file mode 100644 index 0000000000..fa201bbc2f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/MulticastConfigurationFactoryBean.java @@ -0,0 +1,60 @@ +/* + * 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.cluster.protocol.spring; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.factory.FactoryBean; + +/** + * Factory bean for creating a singleton MulticastConfiguration instance. + */ +public class MulticastConfigurationFactoryBean implements FactoryBean { + + private MulticastConfiguration configuration; + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if(configuration == null) { + configuration = new MulticastConfiguration(); + + final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS); + configuration.setSocketTimeout(timeout); + configuration.setReuseAddress(true); + } + return configuration; + + } + + @Override + public Class getObjectType() { + return MulticastConfiguration.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java new file mode 100644 index 0000000000..5b5816d6cc --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/ServerSocketConfigurationFactoryBean.java @@ -0,0 +1,65 @@ +/* + * 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.cluster.protocol.spring; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.io.socket.SSLContextFactory; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; + +import org.springframework.beans.factory.FactoryBean; + +/** + * Factory bean for creating a singleton ServerSocketConfiguration instance. + */ +public class ServerSocketConfigurationFactoryBean implements FactoryBean { + private ServerSocketConfiguration configuration; + private NiFiProperties properties; + + @Override + public ServerSocketConfiguration getObject() throws Exception { + if(configuration == null) { + configuration = new ServerSocketConfiguration(); + configuration.setNeedClientAuth(properties.getNeedClientAuth()); + + final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS); + configuration.setSocketTimeout(timeout); + configuration.setReuseAddress(true); + if(Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) { + configuration.setSSLContextFactory(new SSLContextFactory(properties)); + } + } + return configuration; + + } + + @Override + public Class getObjectType() { + return ServerSocketConfiguration.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java new file mode 100644 index 0000000000..b438e441ae --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/java/org/apache/nifi/cluster/protocol/spring/SocketConfigurationFactoryBean.java @@ -0,0 +1,66 @@ +/* + * 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.cluster.protocol.spring; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.io.socket.SSLContextFactory; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; + +import org.springframework.beans.factory.FactoryBean; + +/** + * Factory bean for creating a singleton SocketConfiguration instance. + */ +public class SocketConfigurationFactoryBean implements FactoryBean { + + private SocketConfiguration configuration; + + private NiFiProperties properties; + + @Override + public SocketConfiguration getObject() throws Exception { + if(configuration == null) { + configuration = new SocketConfiguration(); + + final int timeout = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolSocketTimeout(), TimeUnit.MILLISECONDS); + configuration.setSocketTimeout(timeout); + configuration.setReuseAddress(true); + if(Boolean.valueOf(properties.getProperty(NiFiProperties.CLUSTER_PROTOCOL_IS_SECURE))) { + configuration.setSSLContextFactory(new SSLContextFactory(properties)); + } + } + return configuration; + + } + + @Override + public Class getObjectType() { + return SocketConfiguration.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml new file mode 100644 index 0000000000..07ea7a4de8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/main/resources/nifi-cluster-protocol-context.xml @@ -0,0 +1,110 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java new file mode 100644 index 0000000000..59837c1ae1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java @@ -0,0 +1,134 @@ +/* + * 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.cluster.protocol.impl; + +import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener; +import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl; +import java.io.IOException; +import java.net.InetAddress; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * @author unattributed + */ +public class ClusterManagerProtocolSenderImplTest { + + private InetAddress address; + + private int port; + + private SocketProtocolListener listener; + + private ClusterManagerProtocolSenderImpl sender; + + private ProtocolHandler mockHandler; + + @Before + public void setup() throws IOException { + + address = InetAddress.getLocalHost(); + ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration(); + serverSocketConfiguration.setSocketTimeout(2000); + + mockHandler = mock(ProtocolHandler.class); + + ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext); + listener.addHandler(mockHandler); + listener.start(); + + port = listener.getPort(); + + SocketConfiguration socketConfiguration = new SocketConfiguration(); + sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext); + } + + @After + public void teardown() throws IOException { + if(listener.isRunning()) { + listener.stop(); + } + } + + @Test + public void testRequestFlow() throws Exception { + + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage()); + FlowRequestMessage request = new FlowRequestMessage(); + request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port)); + FlowResponseMessage response = sender.requestFlow(request); + assertNotNull(response); + } + + @Test + public void testRequestFlowWithBadResponseMessage() throws Exception { + + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage()); + FlowRequestMessage request = new FlowRequestMessage(); + request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port)); + try { + sender.requestFlow(request); + fail("failed to throw exception"); + } catch(ProtocolException pe) {} + + } + + @Test + public void testRequestFlowDelayedResponse() throws Exception { + + final int time = 250; + sender.getSocketConfiguration().setSocketTimeout(time); + + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() { + @Override + public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable { + Thread.sleep(time * 3); + return new FlowResponseMessage(); + } + }); + FlowRequestMessage request = new FlowRequestMessage(); + request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port)); + try { + sender.requestFlow(request); + fail("failed to throw exception"); + } catch(ProtocolException pe) {} + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java new file mode 100644 index 0000000000..91f81afc3e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceDiscoveryTest.java @@ -0,0 +1,133 @@ +/* + * 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.cluster.protocol.impl; + +import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastUtils; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +/** + * @author unattributed + */ +public class ClusterServiceDiscoveryTest { + + private ClusterServiceDiscovery discovery; + + private String serviceName; + + private MulticastSocket socket; + + private InetSocketAddress multicastAddress; + + private MulticastConfiguration configuration; + + private ProtocolContext protocolContext; + + @Before + public void setup() throws Exception { + + serviceName = "some-service"; + multicastAddress = new InetSocketAddress("225.1.1.1", 22222); + configuration = new MulticastConfiguration(); + + protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + discovery = new ClusterServiceDiscovery(serviceName, multicastAddress, configuration, protocolContext); + discovery.start(); + + socket = MulticastUtils.createMulticastSocket(multicastAddress.getPort(), configuration); + } + + @After + public void teardown() throws IOException { + try { + if(discovery.isRunning()) { + discovery.stop(); + } + } finally { + MulticastUtils.closeQuietly(socket); + } + } + + @Test + public void testGetAddressOnStartup() { + assertNull(discovery.getService()); + } + + @Ignore("This test has an NPE after ignoring another...perhaps has a bad inter-test dependency") + @Test + public void testGetAddressAfterBroadcast() throws Exception { + + ServiceBroadcastMessage msg = new ServiceBroadcastMessage(); + msg.setServiceName("some-service"); + msg.setAddress("3.3.3.3"); + msg.setPort(1234); + + // marshal message to output stream + ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + marshaller.marshal(msg, baos); + byte[] requestPacketBytes = baos.toByteArray(); + DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress); + socket.send(packet); + + Thread.sleep(250); + + InetSocketAddress updatedAddress = discovery.getService().getServiceAddress(); + assertEquals("some-service", discovery.getServiceName()); + assertEquals("3.3.3.3", updatedAddress.getHostName()); + assertEquals(1234, updatedAddress.getPort()); + + } + + @Test + public void testBadBroadcastMessage() throws Exception { + + ProtocolMessage msg = new PingMessage(); + + // marshal message to output stream + ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + marshaller.marshal(msg, baos); + byte[] requestPacketBytes = baos.toByteArray(); + DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, multicastAddress); + socket.send(packet); + + Thread.sleep(250); + + assertNull(discovery.getService()); + + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java new file mode 100644 index 0000000000..b1c156b506 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java @@ -0,0 +1,121 @@ +/* + * 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.cluster.protocol.impl; + +import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator; +import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery; +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Mockito.*; +import org.mockito.stubbing.OngoingStubbing; + +public class ClusterServiceLocatorTest { + + private ClusterServiceDiscovery mockServiceDiscovery; + + private int fixedPort; + + private DiscoverableService fixedService; + + private ClusterServiceLocator serviceDiscoveryLocator; + + private ClusterServiceLocator serviceDiscoveryFixedPortLocator; + + private ClusterServiceLocator fixedServiceLocator; + + @Before + public void setup() throws Exception { + + fixedPort = 1; + mockServiceDiscovery = mock(ClusterServiceDiscovery.class); + fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20)); + + serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery); + serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort); + fixedServiceLocator = new ClusterServiceLocator(fixedService); + + } + + @Test + public void getServiceWhenServiceDiscoveryNotStarted() { + assertNull(serviceDiscoveryLocator.getService()); + } + + @Test + public void getServiceWhenServiceDiscoveryFixedPortNotStarted() { + assertNull(serviceDiscoveryLocator.getService()); + } + + @Test + public void getServiceWhenFixedServiceNotStarted() { + assertEquals(fixedService, fixedServiceLocator.getService()); + } + + @Test + public void getServiceNotOnFirstAttempt() { + + ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); + config.setNumAttempts(2); + config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); + config.setTimeBetweenAttempts(1); + + serviceDiscoveryLocator.setAttemptsConfig(config); + + OngoingStubbing stubbing = null; + for(int i = 0; i < config.getNumAttempts() - 1; i++) { + if(stubbing == null) { + stubbing = when(mockServiceDiscovery.getService()).thenReturn(null); + } else { + stubbing.thenReturn(null); + } + } + stubbing.thenReturn(fixedService); + + assertEquals(fixedService, serviceDiscoveryLocator.getService()); + + } + + @Test + public void getServiceNotOnFirstAttemptWithFixedPort() { + + ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); + config.setNumAttempts(2); + config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); + config.setTimeBetweenAttempts(1); + + serviceDiscoveryFixedPortLocator.setAttemptsConfig(config); + + OngoingStubbing stubbing = null; + for(int i = 0; i < config.getNumAttempts() - 1; i++) { + if(stubbing == null) { + stubbing = when(mockServiceDiscovery.getService()).thenReturn(null); + } else { + stubbing.thenReturn(null); + } + } + stubbing.thenReturn(fixedService); + + InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort); + DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress); + assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService()); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java new file mode 100644 index 0000000000..ec1f26d2be --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java @@ -0,0 +1,133 @@ +/* + * 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.cluster.protocol.impl; + +import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; +import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener; +import java.net.InetSocketAddress; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +/** + * @author unattributed + */ +public class ClusterServicesBroadcasterTest { + + private ClusterServicesBroadcaster broadcaster; + + private MulticastProtocolListener listener; + + private DummyProtocolHandler handler; + + private InetSocketAddress multicastAddress; + + private DiscoverableService broadcastedService; + + private ProtocolContext protocolContext; + + private MulticastConfiguration configuration; + + @Before + public void setup() throws Exception { + + broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111)); + + multicastAddress = new InetSocketAddress("225.1.1.1", 22222); + + configuration = new MulticastConfiguration(); + + protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms"); + broadcaster.addService(broadcastedService); + + handler = new DummyProtocolHandler(); + listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext); + listener.addHandler(handler); + } + + @After + public void teardown() { + + if(broadcaster.isRunning()) { + broadcaster.stop(); + } + + try { + if(listener.isRunning()) { + listener.stop(); + } + } catch(Exception ex) { + ex.printStackTrace(System.out); + } + + } + + @Ignore("fails needs to be fixed") + @Test + public void testBroadcastReceived() throws Exception { + + broadcaster.start(); + listener.start(); + + Thread.sleep(1000); + + listener.stop(); + + assertNotNull(handler.getProtocolMessage()); + assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType()); + final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage(); + assertEquals(broadcastedService.getServiceName(), msg.getServiceName()); + assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress()); + assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort()); + } + + private class DummyProtocolHandler implements ProtocolHandler { + + private ProtocolMessage protocolMessage; + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + this.protocolMessage = msg; + return null; + } + + public ProtocolMessage getProtocolMessage() { + return protocolMessage; + } + + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java new file mode 100644 index 0000000000..af0059096f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol.impl; + +import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import java.util.ArrayList; +import java.util.List; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastUtils; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +/** + * @author unattributed + */ +public class MulticastProtocolListenerTest { + + private MulticastProtocolListener listener; + + private MulticastSocket socket; + + private InetSocketAddress address; + + private MulticastConfiguration configuration; + + private ProtocolContext protocolContext; + + @Before + public void setup() throws Exception { + + address = new InetSocketAddress("226.1.1.1", 60000); + configuration = new MulticastConfiguration(); + + protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + listener = new MulticastProtocolListener(5, address, configuration, protocolContext); + listener.start(); + + socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration); + } + + @After + public void teardown() throws IOException { + try { + if(listener.isRunning()) { + listener.stop(); + } + } finally { + MulticastUtils.closeQuietly(socket); + } + } + + @Test + public void testBadRequest() throws Exception { + DelayedProtocolHandler handler = new DelayedProtocolHandler(0); + listener.addHandler(handler); + DatagramPacket packet = new DatagramPacket(new byte[] {5}, 1, address); + socket.send(packet); + Thread.sleep(250); + assertEquals(0, handler.getMessages().size()); + } + + @Ignore("this test works sometimes and fails others - needs work to be reliable") + @Test + public void testRequest() throws Exception { + + ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler(); + listener.addHandler(handler); + + ProtocolMessage msg = new PingMessage(); + MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg); + + // marshal message to output stream + ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + marshaller.marshal(multicastMsg, baos); + byte[] requestPacketBytes = baos.toByteArray(); + DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address); + socket.send(packet); + + Thread.sleep(250); + assertEquals(1, handler.getMessages().size()); + assertEquals(msg.getType(), handler.getMessages().get(0).getType()); + + } + + private class ReflexiveProtocolHandler implements ProtocolHandler { + + private List messages = new ArrayList<>(); + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + messages.add(msg); + return msg; + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } + + } + + private class DelayedProtocolHandler implements ProtocolHandler { + + private int delay = 0; + + private List messages = new ArrayList<>(); + + public DelayedProtocolHandler(int delay) { + this.delay = delay; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + try { + messages.add(msg); + Thread.sleep(delay); + return null; + } catch(final InterruptedException ie) { + throw new ProtocolException(ie); + } + + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } + + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java new file mode 100644 index 0000000000..1c5ba9e26a --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java @@ -0,0 +1,203 @@ +/* + * 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.cluster.protocol.impl; + +import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator; +import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener; +import org.apache.nifi.cluster.protocol.impl.NodeProtocolSenderImpl; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.UUID; + +import org.apache.nifi.cluster.protocol.ConnectionRequest; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; + +import org.junit.After; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * @author unattributed + */ +@Ignore("Randomly tests... probably timing-specific") +public class NodeProtocolSenderImplTest { + + private SocketProtocolListener listener; + + private NodeProtocolSenderImpl sender; + + private DiscoverableService service; + + private ServerSocketConfiguration serverSocketConfiguration; + + private ClusterServiceLocator mockServiceLocator; + + private ProtocolHandler mockHandler; + + private NodeIdentifier nodeIdentifier; + + @Before + public void setup() throws IOException { + + serverSocketConfiguration = new ServerSocketConfiguration(); + + mockServiceLocator = mock(ClusterServiceLocator.class); + mockHandler = mock(ProtocolHandler.class); + + nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678); + + ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext); + listener.setShutdownListenerSeconds(3); + listener.addHandler(mockHandler); + listener.start(); + + service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort())); + + SocketConfiguration socketConfiguration = new SocketConfiguration(); + socketConfiguration.setReuseAddress(true); + sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext); + } + + @After + public void teardown() throws IOException { + if(listener.isRunning()) { + listener.stop(); + } + } + + @Test + public void testConnect() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + ConnectionResponseMessage mockMessage = new ConnectionResponseMessage(); + mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier, new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0], new byte[0]), false, null, null, UUID.randomUUID().toString())); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage); + + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + ConnectionResponseMessage response = sender.requestConnection(request); + assertNotNull(response); + } + + @Test(expected = UnknownServiceAddressException.class) + public void testConnectNoClusterManagerAddress() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(null); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage()); + + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + + sender.requestConnection(request); + fail("failed to throw exception"); + } + + @Test(expected = ProtocolException.class) + public void testConnectBadResponse() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage()); + + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + + sender.requestConnection(request); + fail("failed to throw exception"); + + } + + @Test(expected = ProtocolException.class) + public void testConnectDelayedResponse() throws Exception { + + final int time = 250; + sender.getSocketConfiguration().setSocketTimeout(time); + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() { + @Override + public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable { + Thread.sleep(time * 3); + return new ConnectionResponseMessage(); + } + }); + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + + sender.requestConnection(request); + fail("failed to throw exception"); + + } + + @Test + public void testHeartbeat() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null); + + HeartbeatMessage hb = new HeartbeatMessage(); + hb.setHeartbeat(new Heartbeat(new NodeIdentifier("id", "localhost", 3, "localhost", 4), false, false, new byte[] {1, 2, 3})); + sender.heartbeat(hb); + } + + @Test + public void testNotifyControllerStartupFailure() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null); + + ControllerStartupFailureMessage msg = new ControllerStartupFailureMessage(); + msg.setNodeId(new NodeIdentifier("some-id", "some-addr", 1, "some-addr", 1)); + msg.setExceptionMessage("some exception"); + sender.notifyControllerStartupFailure(msg); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.java new file mode 100644 index 0000000000..07ee83ad75 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/DelayedProtocolHandler.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.cluster.protocol.impl.testutils; + +import java.util.ArrayList; +import java.util.List; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; + +/** + * @author unattributed + */ +public class DelayedProtocolHandler implements ProtocolHandler { + + private int delay = 0; + private List messages = new ArrayList<>(); + + public DelayedProtocolHandler(int delay) { + this.delay = delay; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + try { + messages.add(msg); + Thread.sleep(delay); + return null; + } catch (final InterruptedException ie) { + throw new ProtocolException(ie); + } + + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java new file mode 100644 index 0000000000..4e3b9322ec --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-protocol/src/test/java/org/apache/nifi/cluster/protocol/impl/testutils/ReflexiveProtocolHandler.java @@ -0,0 +1,47 @@ +/* + * 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.cluster.protocol.impl.testutils; + +import java.util.ArrayList; +import java.util.List; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; + +/** + * @author unattributed + */ +public class ReflexiveProtocolHandler implements ProtocolHandler { + + private List messages = new ArrayList<>(); + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + messages.add(msg); + return msg; + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-web/.gitignore b/nar-bundles/framework-bundle/framework/cluster-web/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-web/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nar-bundles/framework-bundle/framework/cluster-web/pom.xml b/nar-bundles/framework-bundle/framework/cluster-web/pom.xml new file mode 100644 index 0000000000..a7c39c6be0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-web/pom.xml @@ -0,0 +1,50 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + framework-cluster-web + jar + NiFi Framework Cluster Web + The clustering software for communicating with the NiFi web api. + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-properties + + + org.apache.nifi + web-optimistic-locking + + + org.apache.nifi + nifi-administration + + + org.apache.nifi + nifi-user-actions + + + + diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java new file mode 100644 index 0000000000..44fb25ad97 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContext.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.context; + +import java.io.Serializable; +import java.util.List; +import org.apache.nifi.action.Action; +import org.apache.nifi.web.Revision; + +/** + * Contains contextual information about clustering that may be serialized + * between manager and node when communicating over HTTP. + */ +public interface ClusterContext extends Serializable { + + /** + * Returns a list of auditable actions. The list is modifiable + * and will never be null. + * @return a collection of actions + */ + List getActions(); + + Revision getRevision(); + + void setRevision(Revision revision); + + /** + * @return true if the request was sent by the cluster manager; false otherwise + */ + boolean isRequestSentByClusterManager(); + + /** + * Sets the flag to indicate if a request was sent by the cluster manager. + * @param flag true if the request was sent by the cluster manager; false otherwise + */ + void setRequestSentByClusterManager(boolean flag); + + /** + * Gets an id generation seed. This is used to ensure that nodes are able to generate the + * same id across the cluster. This is usually handled by the cluster manager creating the + * id, however for some actions (snippets, templates, etc) this is not possible. + * @return + */ + String getIdGenerationSeed(); +} diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java new file mode 100644 index 0000000000..06907d2427 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextImpl.java @@ -0,0 +1,69 @@ +/* + * 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.cluster.context; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import org.apache.nifi.action.Action; +import org.apache.nifi.web.Revision; + +/** + * A basic implementation of the context. + */ +public class ClusterContextImpl implements ClusterContext, Serializable { + + private final List actions = new ArrayList<>(); + + private Revision revision; + + private boolean requestSentByClusterManager; + + private final String idGenerationSeed = UUID.randomUUID().toString(); + + @Override + public List getActions() { + return actions; + } + + @Override + public Revision getRevision() { + return revision; + } + + @Override + public void setRevision(Revision revision) { + this.revision = revision; + } + + @Override + public boolean isRequestSentByClusterManager() { + return requestSentByClusterManager; + } + + @Override + public void setRequestSentByClusterManager(boolean requestSentByClusterManager) { + this.requestSentByClusterManager = requestSentByClusterManager; + } + + @Override + public String getIdGenerationSeed() { + return this.idGenerationSeed; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java new file mode 100644 index 0000000000..012e7c7081 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/cluster/context/ClusterContextThreadLocal.java @@ -0,0 +1,47 @@ +/* + * 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.cluster.context; + +/** + * Manages a cluster context on a threadlocal. + */ +public class ClusterContextThreadLocal { + + private static final ThreadLocal contextHolder = new ThreadLocal<>(); + + public static void removeContext() { + contextHolder.remove(); + } + + public static ClusterContext createEmptyContext() { + return new ClusterContextImpl(); + } + + public static ClusterContext getContext() { + ClusterContext ctx = contextHolder.get(); + if(ctx == null) { + ctx = createEmptyContext(); + contextHolder.set(ctx); + } + return ctx; + } + + public static void setContext(final ClusterContext context) { + contextHolder.set(context); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java new file mode 100644 index 0000000000..90b8a370e2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster-web/src/main/java/org/apache/nifi/web/ClusterAwareOptimisticLockingManager.java @@ -0,0 +1,96 @@ +/* + * 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.web; + +import org.apache.nifi.cluster.context.ClusterContext; +import org.apache.nifi.cluster.context.ClusterContextThreadLocal; + +/** + * An optimistic locking manager that provides for optimistic locking in a clustered + * environment. + * + * @author unattributed + */ +public class ClusterAwareOptimisticLockingManager implements OptimisticLockingManager { + + private final OptimisticLockingManager optimisticLockingManager; + + public ClusterAwareOptimisticLockingManager(final OptimisticLockingManager optimisticLockingManager) { + this.optimisticLockingManager = optimisticLockingManager; + } + + @Override + public Revision checkRevision(Revision revision) throws InvalidRevisionException { + final Revision currentRevision = getRevision(); + if(currentRevision.equals(revision) == false) { + throw new InvalidRevisionException(String.format("Given revision %s does not match current revision %s.", revision, currentRevision)); + } else { + return revision.increment(revision.getClientId()); + } + } + + @Override + public boolean isCurrent(Revision revision) { + return getRevision().equals(revision); + } + + @Override + public Revision getRevision() { + final ClusterContext ctx = ClusterContextThreadLocal.getContext(); + if(ctx == null || ctx.getRevision() == null) { + return optimisticLockingManager.getRevision(); + } else { + return ctx.getRevision(); + } + } + + @Override + public void setRevision(final Revision revision) { + final ClusterContext ctx = ClusterContextThreadLocal.getContext(); + if(ctx != null) { + ctx.setRevision(revision); + } + optimisticLockingManager.setRevision(revision); + } + + @Override + public Revision incrementRevision() { + final Revision currentRevision = getRevision(); + final Revision incRevision = currentRevision.increment(); + setRevision(incRevision); + return incRevision; + } + + @Override + public Revision incrementRevision(final String clientId) { + final Revision currentRevision = getRevision(); + final Revision incRevision = currentRevision.increment(clientId); + setRevision(incRevision); + return incRevision; + } + + @Override + public String getLastModifier() { + return optimisticLockingManager.getLastModifier(); + } + + @Override + public void setLastModifier(final String lastModifier) { + optimisticLockingManager.setLastModifier(lastModifier); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/.gitignore b/nar-bundles/framework-bundle/framework/cluster/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nar-bundles/framework-bundle/framework/cluster/pom.xml b/nar-bundles/framework-bundle/framework/cluster/pom.xml new file mode 100644 index 0000000000..ad5dda73ae --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/pom.xml @@ -0,0 +1,133 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + framework-cluster + jar + NiFi Framework Cluster + The clustering software for NiFi. + + + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-properties + + + org.apache.nifi + nifi-logging-utils + + + org.apache.nifi + nifi-file-utils + + + org.apache.nifi + client-dto + + + org.apache.nifi + framework-core + + + org.apache.nifi + core-api + + + org.apache.nifi + framework-cluster-protocol + + + org.apache.nifi + framework-cluster-web + ${project.version} + + + org.apache.nifi + nifi-web-utils + + + org.apache.nifi + nifi-administration + + + org.apache.nifi + site-to-site + + + org.apache.commons + commons-compress + 1.9 + + + + + + + javax.servlet + javax.servlet-api + + + + + commons-io + commons-io + + + commons-net + commons-net + + + + + com.sun.jersey + jersey-client + + + com.sun.jersey + jersey-server + + + com.sun.jersey + jersey-json + + + + + org.springframework + spring-core + + + org.springframework + spring-beans + + + org.springframework + spring-context + + + + diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java new file mode 100644 index 0000000000..183c7ca9b5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/client/MulticastTestClient.java @@ -0,0 +1,149 @@ +/* + * 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.cluster.client; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import java.text.SimpleDateFormat; +import java.util.Date; +import java.util.Timer; +import java.util.TimerTask; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.impl.MulticastProtocolListener; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Simple multicast test client that sends ping messages to a group address. + */ +public class MulticastTestClient { + + private static final Logger logger = LoggerFactory.getLogger(MulticastTestClient.class); + + private static final int PING_DELAY_SECONDS = 3; + + public static void main(final String... args) throws IOException { + + String group = System.getProperty("group", "225.0.0.0"); + if (group == null) { + System.out.println("Host system property 'group' was not given."); + return; + } + group = group.trim(); + if (group.length() == 0) { + System.out.println("Host system property 'group' must be non-empty."); + return; + } + + final String portStr = System.getProperty("port", "2222"); + final int port; + try { + port = Integer.parseInt(portStr); + } catch (final NumberFormatException nfe) { + System.out.println("Port system property 'port' was not a valid port."); + return; + } + + logger.info(String.format("Pinging every %s seconds using multicast address: %s:%s.", PING_DELAY_SECONDS, group, port)); + logger.info("Override defaults by using system properties '-Dgroup=' and '-Dport='."); + logger.info("The test client may be stopped by entering a newline at the command line."); + + final InetSocketAddress addr = new InetSocketAddress(group, port); + final ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + final MulticastConfiguration multicastConfig = new MulticastConfiguration(); + multicastConfig.setReuseAddress(true); + + // setup listener + final MulticastProtocolListener listener = new MulticastProtocolListener(1, addr, multicastConfig, protocolContext); + listener.addHandler(new ProtocolHandler() { + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + final PingMessage pingMsg = (PingMessage) msg; + final SimpleDateFormat sdf = new SimpleDateFormat("HH:mm:ss"); + logger.info("Pinged at: " + sdf.format(pingMsg.getDate())); + return null; + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + }); + + // setup socket + final MulticastSocket multicastSocket = MulticastUtils.createMulticastSocket(multicastConfig); + + // setup broadcaster + final Timer broadcaster = new Timer("Multicast Test Client", /** + * is daemon * + */ + true); + + try { + + // start listening + listener.start(); + + // start broadcasting + broadcaster.schedule(new TimerTask() { + + @Override + public void run() { + try { + + final PingMessage msg = new PingMessage(); + msg.setDate(new Date()); + + // marshal message to output stream + final ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + marshaller.marshal(msg, baos); + final byte[] packetBytes = baos.toByteArray(); + + // send message + final DatagramPacket packet = new DatagramPacket(packetBytes, packetBytes.length, addr); + multicastSocket.send(packet); + + } catch (final Exception ex) { + logger.warn("Failed to send message due to: " + ex, ex); + } + } + }, 0, PING_DELAY_SECONDS * 1000); + + // block until any input is received + System.in.read(); + + } finally { + broadcaster.cancel(); + if (listener.isRunning()) { + listener.stop(); + } + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java new file mode 100644 index 0000000000..6bc5d6cc35 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/Event.java @@ -0,0 +1,122 @@ +/* + * 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.cluster.event; + +import java.util.Date; +import org.apache.commons.lang3.StringUtils; + +/** + * Events describe the occurrence of something noteworthy. They record the + * event's source, a timestamp, a description, and a category. + * + * @author unattributed + * + * @Immutable + */ +public class Event { + + public static enum Category { + + DEBUG, + INFO, + WARN + } + + private final String source; + + private final long timestamp; + + private final Category category; + + private final String message; + + /** + * Creates an event with the current time as the timestamp and a category of + * "INFO". + * + * @param source the source + * @param message the description + */ + public Event(final String source, final String message) { + this(source, message, Category.INFO); + } + + /** + * Creates an event with the current time as the timestamp. + * + * @param source the source + * @param message the description + * @param category the event category + */ + public Event(final String source, final String message, final Category category) { + this(source, message, category, new Date().getTime()); + } + + /** + * Creates an event with the a category of "INFO". + * + * @param source the source + * @param message the description + * @param timestamp the time of occurrence + */ + public Event(final String source, final String message, final long timestamp) { + this(source, message, Category.INFO, timestamp); + } + + /** + * Creates an event. + * + * @param source the source + * @param message the description + * @param category the event category + * @param timestamp the time of occurrence + */ + public Event(final String source, final String message, final Category category, final long timestamp) { + + if (StringUtils.isBlank(source)) { + throw new IllegalArgumentException("Source may not be empty or null."); + } else if (StringUtils.isBlank(message)) { + throw new IllegalArgumentException("Event message may not be empty or null."); + } else if (category == null) { + throw new IllegalArgumentException("Event category may not be null."); + } else if (timestamp < 0) { + throw new IllegalArgumentException("Timestamp may not be negative: " + timestamp); + } + + this.source = source; + this.message = message; + this.category = category; + this.timestamp = timestamp; + } + + public Category getCategory() { + return category; + } + + public String getMessage() { + return message; + } + + public String getSource() { + return source; + } + + public long getTimestamp() { + return timestamp; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java new file mode 100644 index 0000000000..f9dfb00fe1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/EventManager.java @@ -0,0 +1,65 @@ +/* + * 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.cluster.event; + +import java.util.List; + +/** + * Manages an ordered list of events. The event history size dictates the total + * number of events to manage for a given source at a given time. When the size + * is exceeded, the oldest event for that source is evicted. + * + * @author unattributed + */ +public interface EventManager { + + /** + * Adds an event to the manager. + * + * @param event an Event + */ + void addEvent(Event event); + + /** + * Returns a list of events for a given source sorted by the event's + * timestamp where the most recent event is first in the list. + * + * @param eventSource the source + * + * @return the list of events + */ + List getEvents(String eventSource); + + /* + * Returns the most recent event for the source. If no events exist, then + * null is returned. + */ + Event getMostRecentEvent(String eventSource); + + /* + * Clears all events for the given source. + */ + void clearEventHistory(String eventSource); + + /** + * Returns the history size. + * + * @return the history size + */ + int getEventHistorySize(); + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java new file mode 100644 index 0000000000..7fadc78ffa --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/event/impl/EventManagerImpl.java @@ -0,0 +1,143 @@ +/* + * 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.cluster.event.impl; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Queue; +import org.apache.nifi.cluster.event.Event; +import org.apache.nifi.cluster.event.EventManager; + +/** + * Implements the EventManager. + * + * @author unattributed + */ +public class EventManagerImpl implements EventManager { + + /** + * associates the source ID with an ordered queue of events, ordered by most + * recent event + */ + private final Map> eventsMap = new HashMap<>(); + + /** + * the number of events to maintain for a given source + */ + private final int eventHistorySize; + + /** + * Creates an instance. + * + * @param eventHistorySize the number of events to manage for a given + * source. Value must be positive. + */ + public EventManagerImpl(final int eventHistorySize) { + if (eventHistorySize <= 0) { + throw new IllegalArgumentException("Event history size must be positive: " + eventHistorySize); + } + this.eventHistorySize = eventHistorySize; + } + + @Override + public void addEvent(final Event event) { + + if (event == null) { + throw new IllegalArgumentException("Event may not be null."); + } + + Queue events = eventsMap.get(event.getSource()); + if (events == null) { + // no events from this source, so add a new queue to the map + events = new PriorityQueue<>(eventHistorySize, createEventComparator()); + eventsMap.put(event.getSource(), events); + } + + // add event + events.add(event); + + // if we exceeded the history size, then evict the oldest event + if (events.size() > eventHistorySize) { + removeOldestEvent(events); + } + + } + + @Override + public List getEvents(final String eventSource) { + final Queue events = eventsMap.get(eventSource); + if (events == null) { + return Collections.EMPTY_LIST; + } else { + return Collections.unmodifiableList(new ArrayList<>(events)); + } + } + + @Override + public int getEventHistorySize() { + return eventHistorySize; + } + + @Override + public Event getMostRecentEvent(final String eventSource) { + final Queue events = eventsMap.get(eventSource); + if (events == null) { + return null; + } else { + return events.peek(); + } + } + + @Override + public void clearEventHistory(final String eventSource) { + eventsMap.remove(eventSource); + } + + private Comparator createEventComparator() { + return new Comparator() { + @Override + public int compare(final Event o1, final Event o2) { + // orders events by most recent first + return (int) (o2.getTimestamp() - o1.getTimestamp()); + } + }; + } + + private void removeOldestEvent(final Collection events) { + + if (events.isEmpty()) { + return; + } + + Event oldestEvent = null; + for (final Event event : events) { + if (oldestEvent == null || oldestEvent.getTimestamp() > event.getTimestamp()) { + oldestEvent = event; + } + } + + events.remove(oldestEvent); + + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java new file mode 100644 index 0000000000..2e3d2784d7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/ClusterNodeFirewall.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.firewall; + +/** + * Defines the interface for restricting external client connections to a set of + * hosts or IPs. + */ +public interface ClusterNodeFirewall { + + /** + * Returns true if the given host or IP is permissible through the firewall; + * false otherwise. + * + * If an IP is given, then it must be formatted in dotted decimal notation. + * @param hostOrIp + * @return + */ + boolean isPermissible(String hostOrIp); + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java new file mode 100644 index 0000000000..bcee661b0c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewall.java @@ -0,0 +1,207 @@ +/* + * 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.cluster.firewall.impl; + +import java.io.*; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.*; +import org.apache.commons.net.util.SubnetUtils; +import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; +import org.apache.nifi.file.FileUtils; +import org.apache.nifi.logging.NiFiLog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A file-based implementation of the ClusterFirewall interface. The class is + * configured with a file. If the file is empty, then everything is permissible. + * Otherwise, the file should contain hostnames or IPs formatted as dotted + * decimals with an optional CIDR suffix. Each entry must be separated by a + * newline. An example configuration is given below: + * + * + * # hash character is a comment delimiter + * 1.2.3.4 # exact IP + * some.host.name # a host name + * 4.5.6.7/8 # range of CIDR IPs + * 9.10.11.12/13 # a smaller range of CIDR IPs + * + * + * This class allows for synchronization with an optionally configured restore + * directory. If configured, then at startup, if the either the config file or + * the restore directory's copy is missing, then the configuration file will be + * copied to the appropriate location. If both restore directory contains a copy + * that is different in content to configuration file, then an exception is + * thrown at construction time. + */ +public class FileBasedClusterNodeFirewall implements ClusterNodeFirewall { + + private final File config; + + private final File restoreDirectory; + + private final Collection subnetInfos = new ArrayList<>(); + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(FileBasedClusterNodeFirewall.class)); + + public FileBasedClusterNodeFirewall(final File config) throws IOException { + this(config, null); + } + + public FileBasedClusterNodeFirewall(final File config, final File restoreDirectory) throws IOException { + + if (config == null) { + throw new IllegalArgumentException("Firewall configuration file may not be null."); + } + + this.config = config; + this.restoreDirectory = restoreDirectory; + + if (restoreDirectory != null) { + // synchronize with restore directory + try { + syncWithRestoreDirectory(); + } catch (final IOException ioe) { + throw new RuntimeException(ioe); + } + } + + if (!config.exists() && !config.createNewFile()) { + throw new IOException("Firewall configuration file did not exist and could not be created: " + config.getAbsolutePath()); + } + + logger.info("Loading cluster firewall configuration."); + parseConfig(config); + logger.info("Cluster firewall configuration loaded."); + } + + @Override + public boolean isPermissible(final String hostOrIp) { + try { + + // if no rules, then permit everything + if (subnetInfos.isEmpty()) { + return true; + } + + final String ip; + try { + ip = InetAddress.getByName(hostOrIp).getHostAddress(); + } catch (final UnknownHostException uhe) { + logger.warn("Blocking unknown host: " + hostOrIp, uhe); + return false; + } + + // check each subnet to see if IP is in range + for (final SubnetUtils.SubnetInfo subnetInfo : subnetInfos) { + if (subnetInfo.isInRange(ip)) { + return true; + } + } + + // no match + return false; + + } catch (final IllegalArgumentException iae) { + return false; + } + } + + private void syncWithRestoreDirectory() throws IOException { + + // sanity check that restore directory is a directory, creating it if necessary + FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory); + + // check that restore directory is not the same as the primary directory + if (config.getParentFile().getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) { + throw new IllegalStateException( + String.format("Cluster firewall configuration file '%s' cannot be in the restore directory '%s' ", + config.getAbsolutePath(), restoreDirectory.getAbsolutePath())); + } + + // the restore copy will have same file name, but reside in a different directory + final File restoreFile = new File(restoreDirectory, config.getName()); + + // sync the primary copy with the restore copy + FileUtils.syncWithRestore(config, restoreFile, logger); + + } + + private void parseConfig(final File config) throws IOException { + + // clear old information + subnetInfos.clear(); + try (BufferedReader br = new BufferedReader(new FileReader(config))) { + + String ipOrHostLine; + String ipCidr; + int totalIpsAdded = 0; + while ((ipOrHostLine = br.readLine()) != null) { + + // cleanup whitespace + ipOrHostLine = ipOrHostLine.trim(); + + if (ipOrHostLine.isEmpty() || ipOrHostLine.startsWith("#")) { + // skip empty lines or comments + continue; + } else if (ipOrHostLine.contains("#")) { + // parse out comments in IP containing lines + ipOrHostLine = ipOrHostLine.substring(0, ipOrHostLine.indexOf("#")).trim(); + } + + // if given a complete IP, then covert to CIDR + if (ipOrHostLine.contains("/")) { + ipCidr = ipOrHostLine; + } else if (ipOrHostLine.contains("\\")) { + logger.warn("CIDR IP notation uses forward slashes '/'. Replacing backslash '\\' with forward slash'/' for '" + ipOrHostLine + "'"); + ipCidr = ipOrHostLine.replace("\\", "/"); + } else { + try { + ipCidr = InetAddress.getByName(ipOrHostLine).getHostAddress(); + if (!ipOrHostLine.equals(ipCidr)) { + logger.debug(String.format("Resolved host '%s' to ip '%s'", ipOrHostLine, ipCidr)); + } + ipCidr += "/32"; + logger.debug("Adding CIDR to exact IP: " + ipCidr); + } catch (final UnknownHostException uhe) { + logger.warn("Firewall is skipping unknown host address: " + ipOrHostLine); + continue; + } + } + + try { + logger.debug("Adding CIDR IP to firewall: " + ipCidr); + final SubnetUtils subnetUtils = new SubnetUtils(ipCidr); + subnetUtils.setInclusiveHostCount(true); + subnetInfos.add(subnetUtils.getInfo()); + totalIpsAdded++; + } catch (final IllegalArgumentException iae) { + logger.warn("Firewall is skipping invalid CIDR address: " + ipOrHostLine); + } + + } + + if (totalIpsAdded == 0) { + logger.info("No IPs added to firewall. Firewall will accept all requests."); + } else { + logger.info(String.format("Added %d IP(s) to firewall. Only requests originating from the configured IPs will be accepted.", totalIpsAdded)); + } + + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java new file mode 100644 index 0000000000..eedb88fb80 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/ClusterDataFlow.java @@ -0,0 +1,45 @@ +/* + * 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.cluster.flow; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; + +/** + * A dataflow with additional information about the cluster. + * + * @author unattributed + */ +public class ClusterDataFlow { + + private final StandardDataFlow dataFlow; + + private final NodeIdentifier primaryNodeId; + + public ClusterDataFlow(final StandardDataFlow dataFlow, final NodeIdentifier primaryNodeId) { + this.dataFlow = dataFlow; + this.primaryNodeId = primaryNodeId; + } + + public NodeIdentifier getPrimaryNodeId() { + return primaryNodeId; + } + + public StandardDataFlow getDataFlow() { + return dataFlow; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java new file mode 100644 index 0000000000..6ff15a7c31 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DaoException.java @@ -0,0 +1,40 @@ +/* + * 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.cluster.flow; + +/** + * A base exception for data access exceptions. + * + * @author unattributed + */ +public class DaoException extends RuntimeException { + + public DaoException() { + } + + public DaoException(String msg) { + super(msg); + } + + public DaoException(Throwable cause) { + super(cause); + } + + public DaoException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java new file mode 100644 index 0000000000..a2737045d2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowDao.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.flow; + +/** + * A data access object for loading and saving the flow managed by the cluster. + * + * @author unattributed + */ +public interface DataFlowDao { + + /** + * Loads the cluster's dataflow. + * + * @return the dataflow or null if no dataflow exists + * + * @throws DaoException if the dataflow was unable to be loaded + */ + ClusterDataFlow loadDataFlow() throws DaoException; + + /** + * Saves the cluster's dataflow. + * + * + * @param dataFlow + * @throws DaoException if the dataflow was unable to be saved + */ + void saveDataFlow(ClusterDataFlow dataFlow) throws DaoException; + + /** + * Sets the state of the dataflow. If the dataflow does not exist, then an + * exception is thrown. + * + * @param flowState the state of the dataflow + * + * @throws DaoException if the state was unable to be updated + */ + void setPersistedFlowState(PersistedFlowState flowState) throws DaoException; + + /** + * Gets the state of the dataflow. + * + * @return the state of the dataflow + * + * @throws DaoException if the state was unable to be retrieved + */ + PersistedFlowState getPersistedFlowState() throws DaoException; +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java new file mode 100644 index 0000000000..339d9047f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/DataFlowManagementService.java @@ -0,0 +1,115 @@ +/* + * 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.cluster.flow; + +import java.util.Set; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * A service for managing the cluster's flow. The service will attempt to keep + * the cluster's dataflow current while respecting the value of the configured + * retrieval delay. + * + * The eligible retrieval time is reset with the configured delay every time the + * flow state is set to STALE. If the state is set to UNKNOWN or CURRENT, then + * the flow will not be retrieved. + * + * Clients must call start() and stop() to initialize and stop the instance. + * + * @author unattributed + */ +public interface DataFlowManagementService { + + /** + * Starts the instance. Start may only be called if the instance is not + * running. + */ + void start(); + + /** + * Stops the instance. Stop may only be called if the instance is running. + */ + void stop(); + + /** + * @return true if the instance is started; false otherwise. + */ + boolean isRunning(); + + /** + * Loads the dataflow. + * + * @return the dataflow or null if no dataflow exists + */ + ClusterDataFlow loadDataFlow(); + + /** + * Updates the dataflow with the given primary node identifier. + * + * @param nodeId the node identifier + * + * @throws DaoException if the update failed + */ + void updatePrimaryNode(NodeIdentifier nodeId) throws DaoException; + + /** + * Sets the state of the flow. + * + * @param flowState the state + * + * @see PersistedFlowState + */ + void setPersistedFlowState(PersistedFlowState flowState); + + /** + * @return the state of the flow + */ + PersistedFlowState getPersistedFlowState(); + + /** + * @return true if the flow is current; false otherwise. + */ + boolean isFlowCurrent(); + + /** + * Sets the node identifiers to use when attempting to retrieve the flow. + * + * @param nodeIds the node identifiers + */ + void setNodeIds(Set nodeIds); + + /** + * Returns the set of node identifiers the service is using to retrieve the + * flow. + * + * @return the set of node identifiers the service is using to retrieve the + * flow. + */ + Set getNodeIds(); + + /** + * @return the retrieval delay in seconds + */ + int getRetrievalDelaySeconds(); + + /** + * Sets the retrieval delay. + * + * @param delay the retrieval delay in seconds + */ + void setRetrievalDelay(String delay); +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java new file mode 100644 index 0000000000..b3afc6ee6f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/PersistedFlowState.java @@ -0,0 +1,37 @@ +/* + * 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.cluster.flow; + +/** + * Represents the various state of a flow managed by the cluster. + * + * The semantics of the values are: + *
    + *
  • CURRENT - the flow is current
  • + *
  • STALE - the flow is not current, but is eligible to be updated.
  • + *
  • UNKNOWN - the flow is not current and is not eligible to be updated. + *
  • + *
+ * + * @author unattributed + */ +public enum PersistedFlowState { + + CURRENT, + STALE, + UNKNOWN +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java new file mode 100644 index 0000000000..ce5a08b50c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/StaleFlowException.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.flow; + +/** + * Represents the exceptional case when a caller is requesting the current flow, + * but a current flow is not available. + * + * @author unattributed + */ +public class StaleFlowException extends RuntimeException { + + public StaleFlowException(String message, Throwable cause) { + super(message, cause); + } + + public StaleFlowException(String message) { + super(message); + } + + public StaleFlowException(Throwable cause) { + super(cause); + } + + public StaleFlowException() { + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java new file mode 100644 index 0000000000..702f081f2d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowDaoImpl.java @@ -0,0 +1,551 @@ +/* + * 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.cluster.flow.impl; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.FilenameFilter; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.UUID; + +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; +import javax.xml.bind.Unmarshaller; +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.transform.OutputKeys; +import javax.xml.transform.Transformer; +import javax.xml.transform.TransformerFactory; +import javax.xml.transform.dom.DOMSource; +import javax.xml.transform.stream.StreamResult; + +import org.apache.nifi.cluster.flow.ClusterDataFlow; +import org.apache.nifi.cluster.flow.DaoException; +import org.apache.nifi.cluster.flow.DataFlowDao; +import org.apache.nifi.cluster.flow.PersistedFlowState; +import org.apache.nifi.cluster.protocol.DataFlow; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.jaxb.message.NodeIdentifierAdapter; +import org.apache.nifi.file.FileUtils; +import org.apache.nifi.io.BufferedInputStream; +import org.apache.nifi.io.BufferedOutputStream; +import org.apache.nifi.io.ByteArrayInputStream; +import org.apache.nifi.io.StreamUtils; +import org.apache.nifi.logging.NiFiLog; + +import org.apache.commons.compress.archivers.tar.TarArchiveEntry; +import org.apache.commons.compress.archivers.tar.TarArchiveInputStream; +import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.Document; +import org.w3c.dom.Element; + +/** + * Implements the FlowDao interface. The implementation tracks the state of the + * dataflow by annotating the filename of the flow state file. Specifically, the + * implementation correlates PersistedFlowState states to filename extensions. + * The correlation is as follows: + *
    + *
  • CURRENT maps to flow.xml
  • + *
  • STALE maps to flow.xml.stale
  • + *
  • UNKNOWN maps to flow.xml.unknown
  • + *
+ * Whenever the flow state changes, the flow state file's name is updated to + * denote its state. + * + * The implementation also provides for a restore directory that may be + * configured for higher availability. At instance creation, if the primary or + * restore directories have multiple flow state files, an exception is thrown. + * If the primary directory has a current flow state file, but the restore + * directory does not, then the primary flow state file is copied to the restore + * directory. If the restore directory has a current flow state file, but the + * primary directory does not, then the restore flow state file is copied to the + * primary directory. If both the primary and restore directories have a current + * flow state file and the files are different, then an exception is thrown. + * + * When the flow state file is saved, it is always saved first to the restore + * directory followed by a save to the primary directory. When the flow state + * file is loaded, a check is made to verify that the primary and restore flow + * state files are both current. If either is not current, then an exception is + * thrown. The primary flow state file is always read when the load method is + * called. + * + * @author unattributed + */ +public class DataFlowDaoImpl implements DataFlowDao { + + private final File primaryDirectory; + private final File restoreDirectory; + private final boolean autoStart; + private final String generatedRootGroupId = UUID.randomUUID().toString(); + + public static final String STALE_EXT = ".stale"; + public static final String UNKNOWN_EXT = ".unknown"; + public static final String FLOW_PACKAGE = "flow.tar"; + public static final String FLOW_XML_FILENAME = "flow.xml"; + public static final String TEMPLATES_FILENAME = "templates.xml"; + public static final String SNIPPETS_FILENAME = "snippets.xml"; + public static final String CLUSTER_INFO_FILENAME = "cluster-info.xml"; + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowDaoImpl.class)); + + public DataFlowDaoImpl(final File primaryDirectory) throws DaoException { + this(primaryDirectory, null, false); + } + + public DataFlowDaoImpl(final File primaryDirectory, final File restoreDirectory, final boolean autoStart) throws DaoException { + + // sanity check that primary directory is a directory, creating it if necessary + if (primaryDirectory == null) { + throw new IllegalArgumentException("Primary directory may not be null."); + } else if (!primaryDirectory.exists()) { + if (!primaryDirectory.mkdir()) { + throw new DaoException(String.format("Failed to create primary directory '%s'", primaryDirectory.getAbsolutePath())); + } + } else if (!primaryDirectory.isDirectory()) { + throw new IllegalArgumentException("Primary directory must be a directory."); + } + + this.autoStart = autoStart; + + try { + this.primaryDirectory = primaryDirectory; + this.restoreDirectory = restoreDirectory; + + if (restoreDirectory == null) { + // check that we have exactly one current flow state file + ensureSingleCurrentStateFile(primaryDirectory); + } else { + + // check that restore directory is a directory, creating it if necessary + FileUtils.ensureDirectoryExistAndCanAccess(restoreDirectory); + + // check that restore directory is not the same as the primary directory + if (primaryDirectory.getAbsolutePath().equals(restoreDirectory.getAbsolutePath())) { + throw new IllegalArgumentException(String.format("Primary directory '%s' is the same as restore directory '%s' ", + primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath())); + } + + final File[] primaryFlowStateFiles = getFlowStateFiles(primaryDirectory); + final File[] restoreFlowStateFiles = getFlowStateFiles(restoreDirectory); + + // if more than one state file in either primary or restore, then throw exception + if (primaryFlowStateFiles.length > 1) { + throw new IllegalStateException(String.format("Found multiple dataflow state files in primary directory '%s'", primaryDirectory)); + } else if (restoreFlowStateFiles.length > 1) { + throw new IllegalStateException(String.format("Found multiple dataflow state files in restore directory '%s'", restoreDirectory)); + } + + // check that the single primary state file we found is current or create a new one + final File primaryFlowStateFile = ensureSingleCurrentStateFile(primaryDirectory); + + // check that the single restore state file we found is current or create a new one + final File restoreFlowStateFile = ensureSingleCurrentStateFile(restoreDirectory); + + // if there was a difference in flow state file directories, then copy the appropriate files + if (restoreFlowStateFiles.length == 0 && primaryFlowStateFiles.length != 0) { + // copy primary state file to restore + FileUtils.copyFile(primaryFlowStateFile, restoreFlowStateFile, false, false, logger); + } else if (primaryFlowStateFiles.length == 0 && restoreFlowStateFiles.length != 0) { + // copy restore state file to primary + FileUtils.copyFile(restoreFlowStateFile, primaryFlowStateFile, false, false, logger); + } else { + // sync the primary copy with the restore copy + FileUtils.syncWithRestore(primaryFlowStateFile, restoreFlowStateFile, logger); + } + + } + } catch (final IOException | IllegalArgumentException | IllegalStateException | JAXBException ex) { + throw new DaoException(ex); + } + + } + + @Override + public ClusterDataFlow loadDataFlow() throws DaoException { + try { + return parseDataFlow(getExistingFlowStateFile(primaryDirectory)); + } catch (final IOException | JAXBException ex) { + throw new DaoException(ex); + } + } + + @Override + public void saveDataFlow(final ClusterDataFlow dataFlow) throws DaoException { + try { + + final File primaryStateFile = getFlowStateFile(primaryDirectory); + + // write to restore before writing to primary in case primary experiences problems + if (restoreDirectory != null) { + final File restoreStateFile = getFlowStateFile(restoreDirectory); + if (restoreStateFile == null) { + if (primaryStateFile == null) { + writeDataFlow(createNewFlowStateFile(restoreDirectory), dataFlow); + } else { + throw new DaoException(String.format("Unable to save dataflow because dataflow state file in primary directory '%s' exists, but it does not exist in the restore directory '%s'", + primaryDirectory.getAbsolutePath(), restoreDirectory.getAbsolutePath())); + } + } else { + if (primaryStateFile == null) { + throw new DaoException(String.format("Unable to save dataflow because dataflow state file in restore directory '%s' exists, but it does not exist in the primary directory '%s'", + restoreDirectory.getAbsolutePath(), primaryDirectory.getAbsolutePath())); + } else { + final PersistedFlowState primaryFlowState = getPersistedFlowState(primaryStateFile); + final PersistedFlowState restoreFlowState = getPersistedFlowState(restoreStateFile); + if (primaryFlowState == restoreFlowState) { + writeDataFlow(restoreStateFile, dataFlow); + } else { + throw new DaoException(String.format("Unable to save dataflow because state file in primary directory '%s' has state '%s', but the state file in the restore directory '%s' has state '%s'", + primaryDirectory.getAbsolutePath(), primaryFlowState, restoreDirectory.getAbsolutePath(), restoreFlowState)); + } + } + } + } + + // write dataflow to primary + if (primaryStateFile == null) { + writeDataFlow(createNewFlowStateFile(primaryDirectory), dataFlow); + } else { + writeDataFlow(primaryStateFile, dataFlow); + } + + } catch (final IOException | JAXBException ex) { + throw new DaoException(ex); + } + } + + @Override + public PersistedFlowState getPersistedFlowState() { + // trust restore over primary if configured for restore + if (restoreDirectory == null) { + return getPersistedFlowState(getExistingFlowStateFile(primaryDirectory)); + } else { + return getPersistedFlowState(getExistingFlowStateFile(restoreDirectory)); + } + } + + @Override + public void setPersistedFlowState(final PersistedFlowState flowState) throws DaoException { + // rename restore before primary if configured for restore + if (restoreDirectory != null) { + renameFlowStateFile(getExistingFlowStateFile(restoreDirectory), flowState); + } + renameFlowStateFile(getExistingFlowStateFile(primaryDirectory), flowState); + } + + private File ensureSingleCurrentStateFile(final File dir) throws IOException, JAXBException { + + // ensure that we have at most one state file and if we have one, it is current + final File[] directoryFlowStateFiles = getFlowStateFiles(dir); + if (directoryFlowStateFiles.length > 1) { + throw new DaoException(String.format("Found multiple dataflow state files in directory '%s'", dir)); + } else if (directoryFlowStateFiles.length == 0) { + // create a new file if none exist + return createNewFlowStateFile(dir); + } else { + // check that the single flow state file is current + final PersistedFlowState flowState = getPersistedFlowState(directoryFlowStateFiles[0]); + if (PersistedFlowState.CURRENT == flowState) { + return directoryFlowStateFiles[0]; + } else { + throw new DaoException(String.format("Dataflow state file '%s' must be current.", directoryFlowStateFiles[0].getAbsolutePath())); + } + } + + } + + private PersistedFlowState getPersistedFlowState(final File file) { + final String path = file.getAbsolutePath(); + if (path.endsWith(STALE_EXT)) { + return PersistedFlowState.STALE; + } else if (path.endsWith(UNKNOWN_EXT)) { + return PersistedFlowState.UNKNOWN; + } else { + return PersistedFlowState.CURRENT; + } + } + + private File getFlowStateFile(final File dir) { + final File[] files = getFlowStateFiles(dir); + if (files.length > 1) { + throw new IllegalStateException(String.format("Expected at most one dataflow state file, but found %s files.", files.length)); + } else if (files.length == 0) { + return null; + } else { + return files[0]; + } + } + + private File getExistingFlowStateFile(final File dir) { + final File file = getFlowStateFile(dir); + if (file == null) { + throw new IllegalStateException(String.format("Expected a dataflow state file, but none existed in directory '%s'", dir.getAbsolutePath())); + } + return file; + } + + private File[] getFlowStateFiles(final File dir) { + final File[] files = dir.listFiles(new FilenameFilter() { + @Override + public boolean accept(File dir, String name) { + return (name.equals(FLOW_PACKAGE) || name.endsWith(STALE_EXT) || name.endsWith(UNKNOWN_EXT)); + } + }); + + if (files == null) { + return new File[0]; + } else { + return files; + } + } + + private File removeStateFileExtension(final File file) { + + final String path = file.getAbsolutePath(); + final int stateFileExtIndex; + if (path.endsWith(STALE_EXT)) { + stateFileExtIndex = path.lastIndexOf(STALE_EXT); + } else if (path.endsWith(UNKNOWN_EXT)) { + stateFileExtIndex = path.lastIndexOf(UNKNOWN_EXT); + } else { + stateFileExtIndex = path.length(); + } + + return new File(path.substring(0, stateFileExtIndex)); + } + + private File addStateFileExtension(final File file, final PersistedFlowState state) { + switch (state) { + case CURRENT: { + return file; + } + case STALE: { + return new File(file.getAbsolutePath() + STALE_EXT); + } + case UNKNOWN: { + return new File(file.getAbsolutePath() + UNKNOWN_EXT); + } + default: { + throw new RuntimeException("Unsupported PersistedFlowState Enum value: " + state); + } + } + } + + private File createNewFlowStateFile(final File dir) throws IOException, JAXBException { + final File stateFile = new File(dir, FLOW_PACKAGE); + stateFile.createNewFile(); + + final byte[] flowBytes = getEmptyFlowBytes(); + final byte[] templateBytes = new byte[0]; + final byte[] snippetBytes = new byte[0]; + final DataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); + + final ClusterMetadata clusterMetadata = new ClusterMetadata(); + writeDataFlow(stateFile, dataFlow, clusterMetadata); + + return stateFile; + } + + private byte[] getEmptyFlowBytes() throws IOException { + try { + final DocumentBuilder docBuilder = DocumentBuilderFactory.newInstance().newDocumentBuilder(); + final Document document = docBuilder.newDocument(); + + final Element controller = document.createElement("flowController"); + document.appendChild(controller); + + controller.appendChild(createTextElement(document, "maxThreadCount", "15")); + + final Element rootGroup = document.createElement("rootGroup"); + rootGroup.appendChild(createTextElement(document, "id", generatedRootGroupId)); + rootGroup.appendChild(createTextElement(document, "name", "NiFi Flow")); + + // create the position element + final Element positionElement = createTextElement(document, "position", ""); + positionElement.setAttribute("x", "0.0"); + positionElement.setAttribute("y", "0.0"); + rootGroup.appendChild(positionElement); + + rootGroup.appendChild(createTextElement(document, "comment", "")); + controller.appendChild(rootGroup); + + final Transformer transformer = TransformerFactory.newInstance().newTransformer(); + transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2"); + transformer.setOutputProperty(OutputKeys.INDENT, "yes"); + + final DOMSource source = new DOMSource(document); + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + final StreamResult result = new StreamResult(baos); + transformer.transform(source, result); + + return baos.toByteArray(); + } catch (final Exception e) { + throw new IOException(e); + } + } + + private Element createTextElement(final Document document, final String elementName, final String value) { + final Element element = document.createElement(elementName); + element.setTextContent(value); + return element; + } + + private void renameFlowStateFile(final File flowStateFile, final PersistedFlowState newState) throws DaoException { + final PersistedFlowState existingState = getPersistedFlowState(flowStateFile); + if (existingState != newState) { + final File newFlowStateFile = addStateFileExtension(removeStateFileExtension(flowStateFile), newState); + if (flowStateFile.renameTo(newFlowStateFile) == false) { + throw new DaoException( + String.format("Failed to rename flow state file '%s' to new name '%s'", flowStateFile.getAbsolutePath(), newFlowStateFile.getAbsolutePath())); + } + } + } + + private ClusterDataFlow parseDataFlow(final File file) throws IOException, JAXBException, DaoException { + byte[] flowBytes = new byte[0]; + byte[] templateBytes = new byte[0]; + byte[] snippetBytes = new byte[0]; + byte[] clusterInfoBytes = new byte[0]; + + try (final InputStream inStream = new FileInputStream(file); + final TarArchiveInputStream tarIn = new TarArchiveInputStream(new BufferedInputStream(inStream))) { + TarArchiveEntry tarEntry; + while ((tarEntry = tarIn.getNextTarEntry()) != null) { + switch (tarEntry.getName()) { + case FLOW_XML_FILENAME: + flowBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, flowBytes, true); + break; + case TEMPLATES_FILENAME: + templateBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, templateBytes, true); + break; + case SNIPPETS_FILENAME: + snippetBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, snippetBytes, true); + break; + case CLUSTER_INFO_FILENAME: + clusterInfoBytes = new byte[(int) tarEntry.getSize()]; + StreamUtils.fillBuffer(tarIn, clusterInfoBytes, true); + break; + default: + throw new DaoException("Found Unexpected file in dataflow configuration: " + tarEntry.getName()); + } + } + } + + final ClusterMetadata clusterMetadata; + if (clusterInfoBytes.length == 0) { + clusterMetadata = null; + } else { + final Unmarshaller clusterMetadataUnmarshaller = ClusterMetadata.jaxbCtx.createUnmarshaller(); + clusterMetadata = (ClusterMetadata) clusterMetadataUnmarshaller.unmarshal(new ByteArrayInputStream(clusterInfoBytes)); + } + + final StandardDataFlow dataFlow = new StandardDataFlow(flowBytes, templateBytes, snippetBytes); + dataFlow.setAutoStartProcessors(autoStart); + + return new ClusterDataFlow(dataFlow, (clusterMetadata == null) ? null : clusterMetadata.getPrimaryNodeId()); + } + + private void writeDataFlow(final File file, final ClusterDataFlow clusterDataFlow) throws IOException, JAXBException { + + // get the data flow + DataFlow dataFlow = clusterDataFlow.getDataFlow(); + + // if no dataflow, then write a new dataflow + if (dataFlow == null) { + dataFlow = new StandardDataFlow(new byte[0], new byte[0], new byte[0]); + } + + // setup the cluster metadata + final ClusterMetadata clusterMetadata = new ClusterMetadata(); + clusterMetadata.setPrimaryNodeId(clusterDataFlow.getPrimaryNodeId()); + + // write to disk + writeDataFlow(file, dataFlow, clusterMetadata); + } + + private void writeTarEntry(final TarArchiveOutputStream tarOut, final String filename, final byte[] bytes) throws IOException { + final TarArchiveEntry flowEntry = new TarArchiveEntry(filename); + flowEntry.setSize(bytes.length); + tarOut.putArchiveEntry(flowEntry); + tarOut.write(bytes); + tarOut.closeArchiveEntry(); + } + + private void writeDataFlow(final File file, final DataFlow dataFlow, final ClusterMetadata clusterMetadata) throws IOException, JAXBException { + + try (final OutputStream fos = new FileOutputStream(file); + final TarArchiveOutputStream tarOut = new TarArchiveOutputStream(new BufferedOutputStream(fos))) { + + writeTarEntry(tarOut, FLOW_XML_FILENAME, dataFlow.getFlow()); + writeTarEntry(tarOut, TEMPLATES_FILENAME, dataFlow.getTemplates()); + writeTarEntry(tarOut, SNIPPETS_FILENAME, dataFlow.getSnippets()); + + final ByteArrayOutputStream baos = new ByteArrayOutputStream(256); + writeClusterMetadata(clusterMetadata, baos); + final byte[] clusterInfoBytes = baos.toByteArray(); + + writeTarEntry(tarOut, CLUSTER_INFO_FILENAME, clusterInfoBytes); + } + } + + private void writeClusterMetadata(final ClusterMetadata clusterMetadata, final OutputStream os) throws IOException, JAXBException { + // write cluster metadata to output stream + final Marshaller marshaller = ClusterMetadata.jaxbCtx.createMarshaller(); + marshaller.setProperty(Marshaller.JAXB_FORMATTED_OUTPUT, true); + marshaller.setProperty(Marshaller.JAXB_FRAGMENT, true); + marshaller.setProperty(Marshaller.JAXB_ENCODING, "UTF-8"); + marshaller.marshal(clusterMetadata, os); + } + + @XmlRootElement(name = "clusterMetadata") + private static class ClusterMetadata { + + private NodeIdentifier primaryNodeId; + + private static final JAXBContext jaxbCtx; + + static { + try { + jaxbCtx = JAXBContext.newInstance(ClusterMetadata.class); + } catch (final JAXBException je) { + throw new RuntimeException(je); + } + } + + @XmlJavaTypeAdapter(NodeIdentifierAdapter.class) + public NodeIdentifier getPrimaryNodeId() { + return primaryNodeId; + } + + public void setPrimaryNodeId(final NodeIdentifier primaryNodeId) { + this.primaryNodeId = primaryNodeId; + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java new file mode 100644 index 0000000000..e135af311f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImpl.java @@ -0,0 +1,356 @@ +/* + * 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.cluster.flow.impl; + +import java.util.Collections; +import java.util.Date; +import java.util.Set; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.CopyOnWriteArraySet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.nifi.cluster.flow.ClusterDataFlow; +import org.apache.nifi.cluster.flow.DaoException; +import org.apache.nifi.cluster.flow.DataFlowDao; +import org.apache.nifi.cluster.flow.DataFlowManagementService; +import org.apache.nifi.cluster.flow.PersistedFlowState; +import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.util.FormatUtils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Implements FlowManagementService interface. The service tries to keep the + * cluster's flow current with regards to the available nodes. + * + * The instance may be configured with a retrieval delay, which will reduce the + * number of retrievals performed by the service at the expense of increasing + * the chances that the service will not be able to provide a current flow to + * the caller. + * + * By default, the service will try to update the flow as quickly as possible. + * Configuring a delay enables a less aggressive retrieval strategy. + * Specifically, the eligible retrieval time is reset every time the flow state + * is set to STALE. If the state is set to UNKNOWN or CURRENT, then the flow + * will not be retrieved. + * + * @author unattributed + */ +public class DataFlowManagementServiceImpl implements DataFlowManagementService { + + /* + * Developer Note: + * + * This class maintains an ExecutorService and a Runnable. + * Although the class is not externally threadsafe, its internals are protected to + * accommodate multithread access between the ExecutorServer and the Runnable. + * + */ + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(DataFlowManagementServiceImpl.class)); + + private final DataFlowDao flowDao; + + private final ClusterManagerProtocolSender sender; + + private final Set nodeIds = new CopyOnWriteArraySet<>(); + + private final AtomicBoolean stopRequested = new AtomicBoolean(false); + + private final AtomicLong lastRetrievalTime = new AtomicLong(-1); + + private Timer flowRetriever; + + private long retrievableAfterTime = 0L; + + private AtomicInteger retrievalDelaySeconds = new AtomicInteger(0); + + private final TimingReentrantLock resourceLock = new TimingReentrantLock(new ReentrantLock()); + + public DataFlowManagementServiceImpl(final DataFlowDao flowDao, final ClusterManagerProtocolSender sender) { + if (flowDao == null) { + throw new IllegalArgumentException("Flow DAO may not be null."); + } else if (sender == null) { + throw new IllegalArgumentException("Cluster Manager Protocol Sender may not be null."); + } + this.flowDao = flowDao; + this.sender = sender; + } + + @Override + public void start() { + + if (isRunning()) { + throw new IllegalArgumentException("Instance is already running."); + } + + // reset stop requested + stopRequested.set(false); + + // setup flow retreiver timer + flowRetriever = new Timer("Flow Management Service", /* is daemon */ true); + flowRetriever.schedule(new FlowRetrieverTimerTask(), 0, 500); + } + + @Override + public boolean isRunning() { + return (flowRetriever != null); + } + + @Override + public void stop() { + + if (isRunning() == false) { + throw new IllegalArgumentException("Instance is already stopped."); + } + + // record stop request + stopRequested.set(true); + + flowRetriever.cancel(); + flowRetriever = null; + + } + + @Override + public ClusterDataFlow loadDataFlow() throws DaoException { + resourceLock.lock(); + try { + return flowDao.loadDataFlow(); + } finally { + resourceLock.unlock("loadDataFlow"); + } + } + + @Override + public void updatePrimaryNode(final NodeIdentifier nodeId) { + resourceLock.lock(); + try { + final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); + + final StandardDataFlow dataFlow; + if (existingClusterDataFlow == null) { + dataFlow = null; + } else { + dataFlow = existingClusterDataFlow.getDataFlow(); + } + + flowDao.saveDataFlow(new ClusterDataFlow(dataFlow, nodeId)); + } finally { + resourceLock.unlock("updatePrimaryNode"); + } + } + + @Override + public PersistedFlowState getPersistedFlowState() { + resourceLock.lock(); + try { + return flowDao.getPersistedFlowState(); + } finally { + resourceLock.unlock("getPersistedFlowState"); + } + } + + @Override + public boolean isFlowCurrent() { + return PersistedFlowState.CURRENT == getPersistedFlowState(); + } + + @Override + public void setPersistedFlowState(final PersistedFlowState flowState) { + // lock to ensure state change and retrievable time update are atomic + resourceLock.lock(); + try { + flowDao.setPersistedFlowState(flowState); + if (PersistedFlowState.STALE == flowState) { + retrievableAfterTime = new Date().getTime() + (getRetrievalDelaySeconds() * 1000); + } else if (PersistedFlowState.UNKNOWN == flowState || PersistedFlowState.CURRENT == flowState) { + retrievableAfterTime = Long.MAX_VALUE; + } + } finally { + resourceLock.unlock("setPersistedFlowState"); + } + } + + @Override + public Set getNodeIds() { + return Collections.unmodifiableSet(nodeIds); + } + + @Override + public void setNodeIds(final Set nodeIds) { + + if (nodeIds == null) { + throw new IllegalArgumentException("Node IDs may not be null."); + } + + resourceLock.lock(); + try { + + if (this.nodeIds.equals(nodeIds)) { + return; + } + + this.nodeIds.clear(); + this.nodeIds.addAll(nodeIds); + + } finally { + resourceLock.unlock("setNodeIds"); + } + + } + + @Override + public int getRetrievalDelaySeconds() { + return retrievalDelaySeconds.get(); + } + + @Override + public void setRetrievalDelay(final String retrievalDelay) { + this.retrievalDelaySeconds.set((int) FormatUtils.getTimeDuration(retrievalDelay, TimeUnit.SECONDS)); + } + + public ClusterManagerProtocolSender getSender() { + return sender; + } + + public long getLastRetrievalTime() { + return lastRetrievalTime.get(); + } + + /** + * A timer task for issuing FlowRequestMessage messages to nodes to retrieve + * an updated flow. + */ + private class FlowRetrieverTimerTask extends TimerTask { + + @Override + public void run() { + + resourceLock.lock(); + try { + // if flow is current, then we're done + if (isFlowCurrent()) { + return; + } + } catch (final Exception ex) { + logger.info("Encountered exception checking if flow is current caused by " + ex, ex); + } finally { + resourceLock.unlock("FlowRetrieverTimerTask - isFlowCurrent"); + } + + final FlowRequestMessage request = new FlowRequestMessage(); + for (final NodeIdentifier nodeId : getNodeIds()) { + try { + // setup request + request.setNodeId(nodeId); + + // record request time + final long requestSentTime = new Date().getTime(); + + resourceLock.lock(); + try { + // sanity checks before making request + if (stopRequested.get()) { // did we receive a stop request + logger.debug("Stopping runnable prematurely because a request to stop was issued."); + return; + } else if (requestSentTime < retrievableAfterTime) { + /* + * Retrievable after time was updated while obtaining + * the lock, so try again later + */ + return; + } + } finally { + resourceLock.unlock("FlowRetrieverTimerTask - check stopRequested"); + } + + // send request + final FlowResponseMessage response = sender.requestFlow(request); + + resourceLock.lock(); + try { + // check if the retrieved flow is still valid + if (requestSentTime > retrievableAfterTime) { + logger.info("Saving retrieved flow."); + + final StandardDataFlow dataFlow = response.getDataFlow(); + final ClusterDataFlow existingClusterDataFlow = flowDao.loadDataFlow(); + final ClusterDataFlow currentClusterDataFlow; + if (existingClusterDataFlow == null) { + currentClusterDataFlow = new ClusterDataFlow(dataFlow, null); + } else { + currentClusterDataFlow = new ClusterDataFlow(dataFlow, existingClusterDataFlow.getPrimaryNodeId()); + } + flowDao.saveDataFlow(currentClusterDataFlow); + flowDao.setPersistedFlowState(PersistedFlowState.CURRENT); + lastRetrievalTime.set(new Date().getTime()); + } + + /* + * Retrievable after time was updated while requesting + * the flow, so try again later. + */ + } finally { + resourceLock.unlock("FlowRetrieverTimerTask - saveDataFlow"); + } + + } catch (final Throwable t) { + logger.info("Encountered exception retrieving flow from node " + nodeId + " caused by " + t, t); + } + } + } + } + + private static class TimingReentrantLock { + + private final Lock lock; + private static final Logger logger = LoggerFactory.getLogger("dataFlowManagementService.lock"); + + private final ThreadLocal lockTime = new ThreadLocal<>(); + + public TimingReentrantLock(final Lock lock) { + this.lock = lock; + } + + public void lock() { + lock.lock(); + lockTime.set(System.nanoTime()); + } + + public void unlock(final String task) { + final long nanosLocked = System.nanoTime() - lockTime.get(); + lock.unlock(); + + final long millisLocked = TimeUnit.MILLISECONDS.convert(nanosLocked, TimeUnit.NANOSECONDS); + if (millisLocked > 100L) { + logger.debug("Lock held for {} milliseconds for task: {}", millisLocked, task); + } + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java new file mode 100644 index 0000000000..0fcac8c18f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/ClusterManager.java @@ -0,0 +1,225 @@ +/* + * 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.cluster.manager; + +import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException; +import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException; +import org.apache.nifi.cluster.manager.exception.UnknownNodeException; +import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException; +import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException; +import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException; +import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException; +import org.apache.nifi.cluster.NodeInformant; +import org.apache.nifi.cluster.event.Event; +import org.apache.nifi.cluster.node.Node; +import org.apache.nifi.cluster.node.Node.Status; +import org.apache.nifi.cluster.protocol.ConnectionRequest; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.controller.status.ProcessGroupStatus; +import org.apache.nifi.diagnostics.SystemDiagnostics; +import org.apache.nifi.reporting.BulletinRepository; + +import java.util.List; +import java.util.Set; + +/** + * Defines the interface for a ClusterManager. The cluster manager is a + * threadsafe centralized manager for a cluster. Members of a cluster are nodes. + * A member becomes a node by issuing a connection request to the manager. The + * manager maintains the set of nodes. Nodes may be disconnected, reconnected, + * and deleted. + * + * Nodes are responsible for sending heartbeats to the manager to indicate their + * liveliness. A manager may disconnect a node if it does not receive a + * heartbeat within a configurable time period. A cluster manager instance may + * be configured with how often to monitor received heartbeats + * (getHeartbeatMonitoringIntervalSeconds()) and the maximum time that may + * elapse between node heartbeats before disconnecting the node + * (getMaxHeartbeatGapSeconds()). + * + * Since only a single node may execute isolated processors, the cluster manager + * maintains the notion of a primary node. The primary node is chosen at cluster + * startup and retains the role until a user requests a different node to be the + * primary node. + * + * @author unattributed + */ +public interface ClusterManager extends NodeInformant { + + /** + * Handles a node's heartbeat. + * + * @param heartbeat a heartbeat + * + */ + void handleHeartbeat(Heartbeat heartbeat); + + /** + * @param statuses the statuses of the nodes + * @return the set of nodes + */ + Set getNodes(Status... statuses); + + /** + * @param nodeId + * @return returns the node with the given identifier or null if node does + * not exist + */ + Node getNode(String nodeId); + + /** + * @param statuses + * @return the set of node identifiers with the given node status + */ + Set getNodeIds(Status... statuses); + + /** + * Deletes the node with the given node identifier. If the given node is the + * primary node, then a subsequent request may be made to the manager to set + * a new primary node. + * + * @param nodeId the node identifier + * @param userDn the Distinguished Name of the user requesting the node be + * deleted from the cluster + * + * @throws UnknownNodeException if the node does not exist + * @throws IllegalNodeDeletionException if the node is not in a disconnected + * state + */ + void deleteNode(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDeletionException; + + /** + * Requests a connection to the cluster. + * + * @param request the request + * + * @return the response + */ + ConnectionResponse requestConnection(ConnectionRequest request); + + /** + * Services reconnection requests for a given node. If the node indicates + * reconnection failure, then the node will be set to disconnected. + * Otherwise, a reconnection request will be sent to the node, initiating + * the connection handshake. + * + * @param nodeId a node identifier + * @param userDn the Distinguished Name of the user requesting the + * reconnection + * + * @throws UnknownNodeException if the node does not exist + * @throws IllegalNodeReconnectionException if the node is not disconnected + */ + void requestReconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeReconnectionException; + + /** + * Requests the node with the given identifier be disconnected. + * + * @param nodeId the node identifier + * @param userDn the Distinguished Name of the user requesting the + * disconnection + * + * @throws UnknownNodeException if the node does not exist + * @throws IllegalNodeDisconnectionException if the node cannot be + * disconnected due to the cluster's state (e.g., node is last connected + * node or node is primary) + * @throws UnknownNodeException if the node does not exist + * @throws IllegalNodeDisconnectionException if the node is not disconnected + * @throws NodeDisconnectionException if the disconnection failed + */ + void requestDisconnection(String nodeId, String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException; + + /** + * @return the time in seconds to wait between successive executions of + * heartbeat monitoring + */ + int getHeartbeatMonitoringIntervalSeconds(); + + /** + * @return the maximum time in seconds that is allowed between successive + * heartbeats of a node before disconnecting the node + */ + int getMaxHeartbeatGapSeconds(); + + /** + * Returns a list of node events for the node with the given identifier. The + * events will be returned in order of most recent to least recent according + * to the creation date of the event. + * + * @param nodeId the node identifier + * + * @return the list of events or an empty list if no node exists with the + * given identifier + */ + List getNodeEvents(final String nodeId); + + /** + * Revokes the primary role from the current primary node and assigns the + * primary role to given given node ID. + * + * If role revocation fails, then the current primary node is set to + * disconnected while retaining the primary role and no role assignment is + * performed. + * + * If role assignment fails, then the given node is set to disconnected and + * is given the primary role. + * + * @param nodeId the node identifier + * @param userDn the Distinguished Name of the user requesting that the + * Primary Node be assigned + * + * @throws UnknownNodeException if the node with the given identifier does + * not exist + * @throws IneligiblePrimaryNodeException if the node with the given + * identifier is not eligible to be the primary node + * @throws PrimaryRoleAssignmentException if the cluster was unable to + * change the primary role to the requested node + */ + void setPrimaryNode(String nodeId, String userDn) throws UnknownNodeException, IneligiblePrimaryNodeException, PrimaryRoleAssignmentException; + + /** + * @return the primary node of the cluster or null if no primary node exists + */ + Node getPrimaryNode(); + + /** + * Returns the bulletin repository. + * + * @return + */ + BulletinRepository getBulletinRepository(); + + /** + * Returns a {@link ProcessGroupStatus} that represents the status of all + * nodes with the given {@link Status}es for the given ProcessGroup id, or + * null if no nodes exist with the given statuses + * + * @param groupId + * @return + */ + ProcessGroupStatus getProcessGroupStatus(String groupId); + + /** + * Returns a merged representation of the System Diagnostics for all nodes + * in the cluster + * + * @return + */ + SystemDiagnostics getSystemDiagnostics(); +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java new file mode 100644 index 0000000000..2cf5812631 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpClusterManager.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager; + +import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.UriConstructionException; +import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException; +import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException; +import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * Extends the ClusterManager interface to define how requests issued to the + * cluster manager are federated to the nodes. Specifically, the HTTP protocol + * is used for communicating requests to the cluster manager and to the nodes. + * + * @author unattributed + */ +public interface HttpClusterManager extends ClusterManager { + + /** + * Federates the HTTP request to all connected nodes in the cluster. The + * given URI's host and port will not be used and instead will be adjusted + * for each node's host and port. The node URIs are guaranteed to be + * constructed before issuing any requests, so if a UriConstructionException + * is thrown, then it is guaranteed that no request was issued. + * + * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) + * @param uri the base request URI (up to, but not including, the query + * string) + * @param parameters the request parameters + * @param headers the request headers + * + * @return the client response + * + * @throws NoConnectedNodesException if no nodes are connected as results of + * the request + * @throws NoResponseFromNodesException if no response could be obtained + * @throws UriConstructionException if there was an issue constructing the + * URIs tailored for each individual node + * @throws ConnectingNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is connecting to the cluster + * @throws DisconnectedNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is disconnected from the cluster + * @throws SafeModeMutableRequestException if the request was a PUT, POST, + * DELETE and a the cluster is in safe mode + */ + NodeResponse applyRequest(String method, URI uri, Map> parameters, Map headers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, + DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; + + /** + * Federates the HTTP request to the nodes specified. The given URI's host + * and port will not be used and instead will be adjusted for each node's + * host and port. The node URIs are guaranteed to be constructed before + * issuing any requests, so if a UriConstructionException is thrown, then it + * is guaranteed that no request was issued. + * + * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) + * @param uri the base request URI (up to, but not including, the query + * string) + * @param parameters the request parameters + * @param headers the request headers + * @param nodeIdentifiers the NodeIdentifier for each node that the request + * should be replaced to + * + * @return the client response + * + * @throws NoConnectedNodesException if no nodes are connected as results of + * the request + * @throws NoResponseFromNodesException if no response could be obtained + * @throws UriConstructionException if there was an issue constructing the + * URIs tailored for each individual node + * @throws ConnectingNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is connecting to the cluster + * @throws DisconnectedNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is disconnected from the cluster + * @throws SafeModeMutableRequestException if the request was a PUT, POST, + * DELETE and a the cluster is in safe mode + */ + NodeResponse applyRequest(String method, URI uri, Map> parameters, Map headers, + Set nodeIdentifiers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, + DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; + + /** + * Federates the HTTP request to all connected nodes in the cluster. The + * given URI's host and port will not be used and instead will be adjusted + * for each node's host and port. The node URIs are guaranteed to be + * constructed before issuing any requests, so if a UriConstructionException + * is thrown, then it is guaranteed that no request was issued. + * + * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) + * @param uri the base request URI (up to, but not including, the query + * string) + * @param entity the HTTP request entity + * @param headers the request headers + * + * @return the client response + * + * @throws NoConnectedNodesException if no nodes are connected as results of + * the request + * @throws NoResponseFromNodesException if no response could be obtained + * @throws UriConstructionException if there was an issue constructing the + * URIs tailored for each individual node + * @throws ConnectingNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is connecting to the cluster + * @throws DisconnectedNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is disconnected from the cluster + * @throws SafeModeMutableRequestException if the request was a PUT, POST, + * DELETE and a the cluster is in safe mode + */ + NodeResponse applyRequest(String method, URI uri, Object entity, Map headers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, + DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; + + /** + * Federates the HTTP request to the nodes specified. The given URI's host + * and port will not be used and instead will be adjusted for each node's + * host and port. The node URIs are guaranteed to be constructed before + * issuing any requests, so if a UriConstructionException is thrown, then it + * is guaranteed that no request was issued. + * + * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD) + * @param uri the base request URI (up to, but not including, the query + * string) + * @param entity the HTTP request entity + * @param headers the request headers + * @param nodeIdentifiers the NodeIdentifier for each node that the request + * should be replaced to + * + * @return the client response + * + * @throws NoConnectedNodesException if no nodes are connected as results of + * the request + * @throws NoResponseFromNodesException if no response could be obtained + * @throws UriConstructionException if there was an issue constructing the + * URIs tailored for each individual node + * @throws ConnectingNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is connecting to the cluster + * @throws DisconnectedNodeMutableRequestException if the request was a PUT, + * POST, DELETE and a node is disconnected from the cluster + * @throws SafeModeMutableRequestException if the request was a PUT, POST, + * DELETE and a the cluster is in safe mode + */ + NodeResponse applyRequest(String method, URI uri, Object entity, Map headers, Set nodeIdentifiers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, + DisconnectedNodeMutableRequestException, SafeModeMutableRequestException; +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java new file mode 100644 index 0000000000..fb57622ddf --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpRequestReplicator.java @@ -0,0 +1,99 @@ +/* + * 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.cluster.manager; + +import org.apache.nifi.cluster.manager.exception.UriConstructionException; +import java.net.URI; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * A service for managing the replication of requests to nodes. It is up to the + * implementing class to decide if requests are sent concurrently or serially. + * + * Clients must call start() and stop() to initialize and shutdown the instance. + * The instance must be started before issuing any replication requests. + * + * @author unattributed + */ +public interface HttpRequestReplicator { + + /** + * Starts the instance for replicating requests. Start may only be called if + * the instance is not running. + */ + void start(); + + /** + * Stops the instance from replicating requests. Stop may only be called if + * the instance is running. + */ + void stop(); + + /** + * @return true if the instance is started; false otherwise. + */ + boolean isRunning(); + + /** + * Requests are sent to each node in the cluster. If the request results in + * an exception, then the NodeResourceResponse will contain the exception. + * + * HTTP DELETE and OPTIONS methods must supply an empty parameters map or + * else and IllegalArgumentException is thrown. + * + * @param nodeIds the node identifiers + * @param method the HTTP method (e.g., GET, POST, PUT, DELETE, HEAD, + * OPTIONS) + * @param uri the base request URI (up to, but not including, the query + * string) + * @param parameters any request parameters + * @param headers any HTTP headers + * + * @return the set of node responses + * + * @throws UriConstructionException if a request for a node failed to be + * constructed from the given prototype URI. If thrown, it is guaranteed + * that no request was sent. + */ + Set replicate(Set nodeIds, String method, URI uri, Map> parameters, Map headers) throws UriConstructionException; + + /** + * Requests are sent to each node in the cluster. If the request results in + * an exception, then the NodeResourceResponse will contain the exception. + * + * HTTP DELETE, GET, HEAD, and OPTIONS methods will throw an + * IllegalArgumentException if used. + * + * @param nodeIds the node identifiers + * @param method the HTTP method (e.g., POST, PUT) + * @param uri the base request URI (up to, but not including, the query + * string) + * @param entity an entity + * @param headers any HTTP headers + * + * @return the set of node responses + * + * @throws UriConstructionException if a request for a node failed to be + * constructed from the given prototype URI. If thrown, it is guaranteed + * that no request was sent. + */ + Set replicate(Set nodeIds, String method, URI uri, Object entity, Map headers) throws UriConstructionException; + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java new file mode 100644 index 0000000000..843a6667db --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/HttpResponseMapper.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.manager; + +import java.net.URI; +import java.util.Map; +import java.util.Set; +import org.apache.nifi.cluster.node.Node.Status; + +/** + * Maps a HTTP response to a node status. + * + * @author unattributed + */ +public interface HttpResponseMapper { + + /** + * Maps a HTTP response to a node response and the corresponding node + * status. + * + * @param requestURI the original request URI + * @param nodeResponses a set of node resource responses + * + * @return a map associating the node response to the node status + */ + Map map(URI requestURI, Set nodeResponses); + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java new file mode 100644 index 0000000000..3f966e579d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/NodeResponse.java @@ -0,0 +1,329 @@ +/* + * 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.cluster.manager; + +import com.sun.jersey.api.client.ClientResponse; +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.net.URI; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +import javax.ws.rs.HttpMethod; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.Response.ResponseBuilder; +import javax.ws.rs.core.Response.Status; +import javax.ws.rs.core.StreamingOutput; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.web.api.entity.Entity; + +import org.apache.commons.io.IOUtils; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Encapsulates a node's response in regards to receiving a external API + * request. + * + * Both the ClientResponse and (server) Response may be obtained from this + * instance. The ClientResponse is stored as it is received from the node. This + * includes the entity input stream. The Response is constructed on demand when + * mapping a ClientResponse to the Response. The ClientResponse to Response + * mapping includes copying the ClientResponse's input stream to the Response. + * Therefore, the getResponse() method should not be called more than once. + * Furthermore, the method should not be called if the caller has already read + * the ClientResponse's input stream. + * + * If a ClientResponse was unable to be created, then a NodeResponse will store + * the Throwable, which may be obtained by calling getThrowable(). + * + * This class overrides hashCode and equals and considers two instances to be + * equal if they have the equal NodeIdentifiers. + * + * @author unattributed + */ +public class NodeResponse { + + private static final Logger logger = LoggerFactory.getLogger(NodeResponse.class); + private final String httpMethod; + private final URI requestUri; + private final ClientResponse clientResponse; + private final NodeIdentifier nodeId; + private final Throwable throwable; + private boolean hasCreatedResponse = false; + private final Entity updatedEntity; + private final long requestDurationNanos; + private final String requestId; + + public NodeResponse(final NodeIdentifier nodeId, final String httpMethod, final URI requestUri, final ClientResponse clientResponse, final long requestDurationNanos, final String requestId) { + if (nodeId == null) { + throw new IllegalArgumentException("Node identifier may not be null."); + } else if (StringUtils.isBlank(httpMethod)) { + throw new IllegalArgumentException("Http method may not be null or empty."); + } else if (requestUri == null) { + throw new IllegalArgumentException("Request URI may not be null."); + } else if (clientResponse == null) { + throw new IllegalArgumentException("ClientResponse may not be null."); + } + this.nodeId = nodeId; + this.httpMethod = httpMethod; + this.requestUri = requestUri; + this.clientResponse = clientResponse; + this.throwable = null; + this.updatedEntity = null; + this.requestDurationNanos = requestDurationNanos; + this.requestId = requestId; + } + + public NodeResponse(final NodeIdentifier nodeId, final String httpMethod, final URI requestUri, final Throwable throwable) { + if (nodeId == null) { + throw new IllegalArgumentException("Node identifier may not be null."); + } else if (StringUtils.isBlank(httpMethod)) { + throw new IllegalArgumentException("Http method may not be null or empty."); + } else if (requestUri == null) { + throw new IllegalArgumentException("Request URI may not be null."); + } else if (throwable == null) { + throw new IllegalArgumentException("Throwable may not be null."); + } + this.nodeId = nodeId; + this.httpMethod = httpMethod; + this.requestUri = requestUri; + this.clientResponse = null; + this.throwable = throwable; + this.updatedEntity = null; + this.requestDurationNanos = -1L; + this.requestId = null; + } + + public NodeResponse(final NodeResponse example, final Entity updatedEntity) { + Objects.requireNonNull(example, "NodeResponse cannot be null"); + Objects.requireNonNull(updatedEntity, "UpdatedEntity cannot be null"); + + this.nodeId = example.nodeId; + this.httpMethod = example.httpMethod; + this.requestUri = example.requestUri; + this.clientResponse = example.clientResponse; + this.throwable = example.throwable; + this.updatedEntity = updatedEntity; + this.requestDurationNanos = example.requestDurationNanos; + this.requestId = null; + } + + public NodeIdentifier getNodeId() { + return nodeId; + } + + public String getHttpMethod() { + return httpMethod; + } + + public URI getRequestUri() { + return requestUri; + } + + /** + * @return the HTTP response status code + */ + public int getStatus() { + if (hasThrowable()) { + /* + * since there is a throwable, there is no client input stream to + * worry about maintaining, so we can call getResponse() method + */ + return getResponse().getStatus(); + } else { + /* + * use client response's status instead of calling getResponse().getStatus() + * so that we don't read the client's input stream as part of creating + * the response in the getResponse() method + */ + return clientResponse.getStatus(); + } + } + + /** + * Returns true if the response status is 2xx, false otherwise. + * + * @return + */ + public boolean is2xx() { + final int statusCode = getStatus(); + return (200 <= statusCode && statusCode <= 299); + } + + /** + * Returns true if the response status is 5xx, false otherwise. + * + * @return + */ + public boolean is5xx() { + final int statusCode = getStatus(); + return (500 <= statusCode && statusCode <= 599); + } + + /** + * Returns null if hasThrowable() is true; otherwise the client's response + * is returned. + * + * The ClientResponse's input stream can only be read once. + * + * @return the client's response + */ + public ClientResponse getClientResponse() { + return clientResponse; + } + + /** + * Creates a Response by mapping the ClientResponse values to it. Since the + * ClientResponse's input stream can only be read once, this method should + * only be called once. Furthermore, the caller should not have already read + * the ClientResponse's input stream. + * + * @return the response + */ + public Response getResponse() { + // if the response encapsulates a throwable, then the input stream is never read and the below warning is irrelevant + if (hasCreatedResponse && !hasThrowable()) { + logger.warn("ClientResponse's input stream has already been read. The created response will not contain this data."); + } + hasCreatedResponse = true; + return createResponse(); + } + + /** + * Returns the throwable or null if no throwable exists. + * + * @return the throwable or null if no throwable exists + */ + public Throwable getThrowable() { + return throwable; + } + + /** + * Returns true if a throwable was thrown and a response was not able to be + * created; false otherwise. + * + * @return true if a throwable was thrown and a response was not able to be + * created; false otherwise + */ + public boolean hasThrowable() { + return getThrowable() != null; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final NodeResponse other = (NodeResponse) obj; + if (this.nodeId != other.nodeId && (this.nodeId == null || !this.nodeId.equals(other.nodeId))) { + return false; + } + return true; + } + + @Override + public int hashCode() { + int hash = 7; + hash = 13 * hash + (this.nodeId != null ? this.nodeId.hashCode() : 0); + return hash; + } + + public long getRequestDuration(final TimeUnit timeUnit) { + return timeUnit.convert(requestDurationNanos, TimeUnit.NANOSECONDS); + } + + public String getRequestId() { + return requestId; + } + + private Response createResponse() { + + // if no client response was created, then generate a 500 response + if (hasThrowable()) { + return Response.status(Status.INTERNAL_SERVER_ERROR).build(); + } + + // set the status + final ResponseBuilder responseBuilder = Response.status(clientResponse.getStatus()); + + // set the headers + for (final String key : clientResponse.getHeaders().keySet()) { + final List values = clientResponse.getHeaders().get(key); + for (final String value : values) { + + if (key.equalsIgnoreCase("transfer-encoding") || key.equalsIgnoreCase("content-length")) { + /* + * do not copy the transfer-encoding header (i.e., chunked encoding) or + * the content-length. Let the outgoing response builder determine it. + */ + continue; + } else if (key.equals("X-ClusterContext")) { + /* + * do not copy the cluster context to the response because + * this information is private and should not be sent to + * the client + */ + continue; + } + responseBuilder.header(key, value); + } + } + + // head requests must not have a message-body in the response + if (!HttpMethod.HEAD.equalsIgnoreCase(httpMethod)) { + + // set the entity + if (updatedEntity == null) { + responseBuilder.entity(new StreamingOutput() { + @Override + public void write(final OutputStream output) throws IOException, WebApplicationException { + BufferedInputStream bis = null; + try { + bis = new BufferedInputStream(clientResponse.getEntityInputStream()); + IOUtils.copy(bis, output); + } finally { + IOUtils.closeQuietly(bis); + } + } + }); + } else { + responseBuilder.entity(updatedEntity); + } + } + + return responseBuilder.build(); + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(); + sb.append("NodeResponse[nodeUri=").append(nodeId.getApiAddress()).append(":").append(nodeId.getApiPort()).append(",") + .append("method=").append(httpMethod) + .append(",URI=").append(requestUri) + .append(",ResponseCode=").append(getStatus()) + .append(",Duration=").append(TimeUnit.MILLISECONDS.convert(requestDurationNanos, TimeUnit.NANOSECONDS)).append(" ms]"); + return sb.toString(); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java new file mode 100644 index 0000000000..49bcd35007 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/BlockedByFirewallException.java @@ -0,0 +1,60 @@ +/* + * 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.cluster.manager.exception; + +import org.apache.nifi.cluster.protocol.NodeIdentifier; + +/** + * + */ +public class BlockedByFirewallException extends ClusterException { + + private final NodeIdentifier nodeId; + private final boolean isExistingNode; + + public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode, String msg, Throwable cause) { + super(msg, cause); + this.nodeId = nodeId; + this.isExistingNode = isExistingNode; + } + + public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode, Throwable cause) { + super(cause); + this.nodeId = nodeId; + this.isExistingNode = isExistingNode; + } + + public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode, String msg) { + super(msg); + this.nodeId = nodeId; + this.isExistingNode = isExistingNode; + } + + public BlockedByFirewallException(NodeIdentifier nodeId, boolean isExistingNode) { + this.nodeId = nodeId; + this.isExistingNode = isExistingNode; + } + + public NodeIdentifier getNodeId() { + return nodeId; + } + + public boolean isExistingNode() { + return isExistingNode; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java new file mode 100644 index 0000000000..3bf9752cc1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ClusterException.java @@ -0,0 +1,40 @@ +/* + * 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.cluster.manager.exception; + +/** + * The base exception class for cluster related exceptions. + * + * @author unattributed + */ +public class ClusterException extends RuntimeException { + + public ClusterException() { + } + + public ClusterException(String msg) { + super(msg); + } + + public ClusterException(Throwable cause) { + super(cause); + } + + public ClusterException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java new file mode 100644 index 0000000000..365b5f0b48 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/ConnectingNodeMutableRequestException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a HTTP request that may change a node's + * dataflow is to be replicated while a node is connecting to the cluster. + * + * @author unattributed + */ +public class ConnectingNodeMutableRequestException extends MutableRequestException { + + public ConnectingNodeMutableRequestException() { + } + + public ConnectingNodeMutableRequestException(String msg) { + super(msg); + } + + public ConnectingNodeMutableRequestException(Throwable cause) { + super(cause); + } + + public ConnectingNodeMutableRequestException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java new file mode 100644 index 0000000000..412a555e32 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/DisconnectedNodeMutableRequestException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a HTTP request that may change a node's + * dataflow is to be replicated while one or more nodes are disconnected. + * + * @author unattributed + */ +public class DisconnectedNodeMutableRequestException extends MutableRequestException { + + public DisconnectedNodeMutableRequestException() { + } + + public DisconnectedNodeMutableRequestException(String msg) { + super(msg); + } + + public DisconnectedNodeMutableRequestException(Throwable cause) { + super(cause); + } + + public DisconnectedNodeMutableRequestException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java new file mode 100644 index 0000000000..6c4e67087f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalClusterStateException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Signals that an operation to be performed on a cluster has been invoked at an + * illegal or inappropriate time. + * + * @author unattributed + */ +public class IllegalClusterStateException extends ClusterException { + + public IllegalClusterStateException() { + } + + public IllegalClusterStateException(String msg) { + super(msg); + } + + public IllegalClusterStateException(Throwable cause) { + super(cause); + } + + public IllegalClusterStateException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java new file mode 100644 index 0000000000..adef62a823 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDeletionException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a deletion request is issued to a node + * that cannot be deleted (e.g., the node is not disconnected). + * + * @author unattributed + */ +public class IllegalNodeDeletionException extends IllegalClusterStateException { + + public IllegalNodeDeletionException() { + } + + public IllegalNodeDeletionException(String msg) { + super(msg); + } + + public IllegalNodeDeletionException(Throwable cause) { + super(cause); + } + + public IllegalNodeDeletionException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java new file mode 100644 index 0000000000..7e61b24f76 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeDisconnectionException.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.manager.exception; + +/** + * Represents the exceptional case when a disconnection request is issued to a + * node that cannot be disconnected (e.g., last node in cluster, node is primary + * node). + * + * @author unattributed + */ +public class IllegalNodeDisconnectionException extends IllegalClusterStateException { + + public IllegalNodeDisconnectionException() { + } + + public IllegalNodeDisconnectionException(String msg) { + super(msg); + } + + public IllegalNodeDisconnectionException(Throwable cause) { + super(cause); + } + + public IllegalNodeDisconnectionException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java new file mode 100644 index 0000000000..96c76bcb35 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IllegalNodeReconnectionException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a reconnection request is issued to a + * node that cannot be reconnected (e.g., the node is not disconnected). + * + * @author unattributed + */ +public class IllegalNodeReconnectionException extends IllegalClusterStateException { + + public IllegalNodeReconnectionException() { + } + + public IllegalNodeReconnectionException(String msg) { + super(msg); + } + + public IllegalNodeReconnectionException(Throwable cause) { + super(cause); + } + + public IllegalNodeReconnectionException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java new file mode 100644 index 0000000000..4b0097a4b8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/IneligiblePrimaryNodeException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when the primary role cannot be assigned to a + * node because the node is ineligible for the role. + * + * @author unattributed + */ +public class IneligiblePrimaryNodeException extends IllegalClusterStateException { + + public IneligiblePrimaryNodeException() { + } + + public IneligiblePrimaryNodeException(String msg) { + super(msg); + } + + public IneligiblePrimaryNodeException(Throwable cause) { + super(cause); + } + + public IneligiblePrimaryNodeException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java new file mode 100644 index 0000000000..d1605874ba --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/MutableRequestException.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.manager.exception; + +/** + * Represents the exceptional case when a HTTP request that may change a node's + * state is to be replicated while the cluster or connected nodes are unable to + * change their state (e.g., a new node is connecting to the cluster). + * + * @author unattributed + */ +public class MutableRequestException extends IllegalClusterStateException { + + public MutableRequestException() { + } + + public MutableRequestException(String msg) { + super(msg); + } + + public MutableRequestException(Throwable cause) { + super(cause); + } + + public MutableRequestException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java new file mode 100644 index 0000000000..8d704b968d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoConnectedNodesException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when the cluster is unable to service a + * request because no nodes are connected. + * + * @author unattributed + */ +public class NoConnectedNodesException extends ClusterException { + + public NoConnectedNodesException() { + } + + public NoConnectedNodesException(String msg) { + super(msg); + } + + public NoConnectedNodesException(Throwable cause) { + super(cause); + } + + public NoConnectedNodesException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java new file mode 100644 index 0000000000..9e17a232dc --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NoResponseFromNodesException.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.manager.exception; + +/** + * Represents the exceptional case when the cluster is unable to service a + * request because no nodes returned a response. When the given request is not + * mutable the nodes are left in their previous state. + * + * @author unattributed + */ +public class NoResponseFromNodesException extends ClusterException { + + public NoResponseFromNodesException() { + } + + public NoResponseFromNodesException(String msg) { + super(msg); + } + + public NoResponseFromNodesException(Throwable cause) { + super(cause); + } + + public NoResponseFromNodesException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java new file mode 100644 index 0000000000..3bd2f4b93f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeDisconnectionException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a disconnection request to a node + * failed. + * + * @author unattributed + */ +public class NodeDisconnectionException extends ClusterException { + + public NodeDisconnectionException() { + } + + public NodeDisconnectionException(String msg) { + super(msg); + } + + public NodeDisconnectionException(Throwable cause) { + super(cause); + } + + public NodeDisconnectionException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java new file mode 100644 index 0000000000..8c40cef283 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/NodeReconnectionException.java @@ -0,0 +1,40 @@ +/* + * 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.cluster.manager.exception; + +/** + * Represents the exceptional case when a reconnection request to a node failed. + * + * @author unattributed + */ +public class NodeReconnectionException extends ClusterException { + + public NodeReconnectionException() { + } + + public NodeReconnectionException(String msg) { + super(msg); + } + + public NodeReconnectionException(Throwable cause) { + super(cause); + } + + public NodeReconnectionException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java new file mode 100644 index 0000000000..403f7a5ce1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/PrimaryRoleAssignmentException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when the cluster is unable to update the + * primary role of a node. + * + * @author unattributed + */ +public class PrimaryRoleAssignmentException extends IllegalClusterStateException { + + public PrimaryRoleAssignmentException() { + } + + public PrimaryRoleAssignmentException(String msg) { + super(msg); + } + + public PrimaryRoleAssignmentException(Throwable cause) { + super(cause); + } + + public PrimaryRoleAssignmentException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java new file mode 100644 index 0000000000..f544f26908 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/SafeModeMutableRequestException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a HTTP request that may change a node's + * dataflow is to be replicated while the cluster is in safe mode. + * + * @author unattributed + */ +public class SafeModeMutableRequestException extends MutableRequestException { + + public SafeModeMutableRequestException() { + } + + public SafeModeMutableRequestException(String msg) { + super(msg); + } + + public SafeModeMutableRequestException(Throwable cause) { + super(cause); + } + + public SafeModeMutableRequestException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java new file mode 100644 index 0000000000..914bb56a84 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UnknownNodeException.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.manager.exception; + +/** + * Represents the exceptional case when a request is made for a node that does + * not exist. + * + * @author unattributed + */ +public class UnknownNodeException extends ClusterException { + + public UnknownNodeException() { + } + + public UnknownNodeException(String msg) { + super(msg); + } + + public UnknownNodeException(Throwable cause) { + super(cause); + } + + public UnknownNodeException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java new file mode 100644 index 0000000000..773d7b58f2 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/exception/UriConstructionException.java @@ -0,0 +1,42 @@ +/* + * 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.cluster.manager.exception; + +/** + * Represents the exceptional case when a URI cannot be constructed from the + * given information. This exception is similar to Java's URISyntaxException + * except that it extends RuntimeException. + * + * @author unattributed + */ +public class UriConstructionException extends RuntimeException { + + public UriConstructionException() { + } + + public UriConstructionException(String msg) { + super(msg); + } + + public UriConstructionException(Throwable cause) { + super(cause); + } + + public UriConstructionException(String msg, Throwable cause) { + super(msg, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java new file mode 100644 index 0000000000..2015530f66 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredEventAccess.java @@ -0,0 +1,135 @@ +/* + * 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.cluster.manager.impl; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.nifi.controller.status.ProcessGroupStatus; +import org.apache.nifi.events.EventReporter; +import org.apache.nifi.provenance.ProvenanceEventBuilder; +import org.apache.nifi.provenance.ProvenanceEventRecord; +import org.apache.nifi.provenance.ProvenanceEventRepository; +import org.apache.nifi.provenance.lineage.ComputeLineageSubmission; +import org.apache.nifi.provenance.search.Query; +import org.apache.nifi.provenance.search.QuerySubmission; +import org.apache.nifi.provenance.search.SearchableField; +import org.apache.nifi.reporting.EventAccess; + +public class ClusteredEventAccess implements EventAccess { + + private final WebClusterManager clusterManager; + + public ClusteredEventAccess(final WebClusterManager clusterManager) { + this.clusterManager = clusterManager; + } + + @Override + public ProcessGroupStatus getControllerStatus() { + return clusterManager.getProcessGroupStatus(WebClusterManager.ROOT_GROUP_ID_ALIAS); + } + + @Override + public List getProvenanceEvents(long arg0, int arg1) throws IOException { + return new ArrayList<>(); + } + + @Override + public ProvenanceEventRepository getProvenanceRepository() { + // NCM doesn't have provenance events, because it doesn't process FlowFiles. + // So we just use a Provenance Event Repository that does nothing. + return new ProvenanceEventRepository() { + @Override + public void close() throws IOException { + } + + @Override + public ProvenanceEventRecord getEvent(long eventId) throws IOException { + return null; + } + + @Override + public List getEvents(long startEventId, int maxEvents) throws IOException { + return new ArrayList<>(); + } + + @Override + public Long getMaxEventId() { + return null; + } + + @Override + public List getSearchableAttributes() { + return new ArrayList<>(); + } + + @Override + public List getSearchableFields() { + return new ArrayList<>(); + } + + @Override + public void registerEvent(final ProvenanceEventRecord event) { + } + + @Override + public void registerEvents(final Iterable events) { + } + + @Override + public ComputeLineageSubmission retrieveLineageSubmission(final String submissionId) { + return null; + } + + @Override + public QuerySubmission retrieveQuerySubmission(final String submissionId) { + return null; + } + + @Override + public ComputeLineageSubmission submitExpandChildren(final long eventId) { + return null; + } + + @Override + public ComputeLineageSubmission submitExpandParents(final long eventId) { + return null; + } + + @Override + public ComputeLineageSubmission submitLineageComputation(final String flowFileUuid) { + return null; + } + + @Override + public QuerySubmission submitQuery(final Query query) { + return null; + } + + @Override + public ProvenanceEventBuilder eventBuilder() { + return null; + } + + @Override + public void initialize(EventReporter eventReporter) throws IOException { + + } + }; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java new file mode 100644 index 0000000000..e546f87e27 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/ClusteredReportingContext.java @@ -0,0 +1,165 @@ +/* + * 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.cluster.manager.impl; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.attribute.expression.language.PreparedQuery; +import org.apache.nifi.attribute.expression.language.Query; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.controller.ControllerServiceLookup; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.status.PortStatus; +import org.apache.nifi.controller.status.ProcessGroupStatus; +import org.apache.nifi.controller.status.ProcessorStatus; +import org.apache.nifi.events.BulletinFactory; +import org.apache.nifi.processor.StandardPropertyValue; +import org.apache.nifi.reporting.Bulletin; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.EventAccess; +import org.apache.nifi.reporting.ReportingContext; +import org.apache.nifi.reporting.Severity; + +public class ClusteredReportingContext implements ReportingContext { + + private final EventAccess eventAccess; + private final BulletinRepository bulletinRepository; + private final ControllerServiceProvider serviceProvider; + private final Map properties; + private final Map preparedQueries; + + public ClusteredReportingContext(final EventAccess eventAccess, final BulletinRepository bulletinRepository, + final Map properties, final ControllerServiceProvider serviceProvider) { + this.eventAccess = eventAccess; + this.bulletinRepository = bulletinRepository; + this.properties = Collections.unmodifiableMap(properties); + this.serviceProvider = serviceProvider; + + preparedQueries = new HashMap<>(); + for (final Map.Entry entry : properties.entrySet()) { + final PropertyDescriptor desc = entry.getKey(); + String value = entry.getValue(); + if (value == null) { + value = desc.getDefaultValue(); + } + + final PreparedQuery pq = Query.prepare(value); + preparedQueries.put(desc, pq); + } + } + + @Override + public EventAccess getEventAccess() { + return eventAccess; + } + + @Override + public BulletinRepository getBulletinRepository() { + return bulletinRepository; + } + + @Override + public Bulletin createBulletin(final String category, final Severity severity, final String message) { + return BulletinFactory.createBulletin(category, severity.name(), message); + } + + @Override + public Bulletin createBulletin(final String componentId, final String category, final Severity severity, final String message) { + final ProcessGroupStatus rootGroupStatus = eventAccess.getControllerStatus(); + final String groupId = findGroupId(rootGroupStatus, componentId); + final String componentName = findComponentName(rootGroupStatus, componentId); + + return BulletinFactory.createBulletin(groupId, componentId, componentName, category, severity.name(), message); + } + + @Override + public Map getProperties() { + return Collections.unmodifiableMap(properties); + } + + @Override + public PropertyValue getProperty(final PropertyDescriptor property) { + final String configuredValue = properties.get(property); + return new StandardPropertyValue(configuredValue == null ? property.getDefaultValue() : configuredValue, serviceProvider, preparedQueries.get(property)); + } + + @Override + public ControllerServiceLookup getControllerServiceLookup() { + return serviceProvider; + } + + String findGroupId(final ProcessGroupStatus groupStatus, final String componentId) { + for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) { + if (procStatus.getId().equals(componentId)) { + return groupStatus.getId(); + } + } + + for (final PortStatus portStatus : groupStatus.getInputPortStatus()) { + if (portStatus.getId().equals(componentId)) { + return groupStatus.getId(); + } + } + + for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) { + if (portStatus.getId().equals(componentId)) { + return groupStatus.getId(); + } + } + + for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) { + final String groupId = findGroupId(childGroup, componentId); + if (groupId != null) { + return groupId; + } + } + + return null; + } + + private String findComponentName(final ProcessGroupStatus groupStatus, final String componentId) { + for (final ProcessorStatus procStatus : groupStatus.getProcessorStatus()) { + if (procStatus.getId().equals(componentId)) { + return procStatus.getName(); + } + } + + for (final PortStatus portStatus : groupStatus.getInputPortStatus()) { + if (portStatus.getId().equals(componentId)) { + return groupStatus.getName(); + } + } + + for (final PortStatus portStatus : groupStatus.getOutputPortStatus()) { + if (portStatus.getId().equals(componentId)) { + return groupStatus.getName(); + } + } + + for (final ProcessGroupStatus childGroup : groupStatus.getProcessGroupStatus()) { + final String componentName = findComponentName(childGroup, componentId); + if (componentName != null) { + return componentName; + } + } + + return null; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java new file mode 100644 index 0000000000..81bb7a75d4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImpl.java @@ -0,0 +1,531 @@ +/* + * 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.cluster.manager.impl; + +import com.sun.jersey.api.client.Client; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.api.client.UniformInterfaceException; +import com.sun.jersey.api.client.WebResource; +import com.sun.jersey.api.client.config.ClientConfig; +import com.sun.jersey.api.client.filter.GZIPContentEncodingFilter; +import com.sun.jersey.core.util.MultivaluedMapImpl; + +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.MultivaluedMap; + +import org.apache.nifi.cluster.manager.HttpRequestReplicator; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.exception.UriConstructionException; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.util.FormatUtils; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An implementation of the HttpRequestReplicator interface. This + * implementation parallelizes the node HTTP requests using the given + * ExecutorService instance. Individual requests may have + * connection and read timeouts set, which may be set during instance + * construction. Otherwise, the default is not to timeout. + * + * If a node protocol scheme is provided during construction, then all requests + * will be replicated using the given scheme. If null is provided as the scheme + * (the default), then the requests will be replicated using the scheme of the + * original URI. + * + * Clients must call start() and stop() to initialize and shutdown the instance. + * The instance must be started before issuing any replication requests. + * + * @author unattributed + */ +public class HttpRequestReplicatorImpl implements HttpRequestReplicator { + + // defaults + private static final int DEFAULT_SHUTDOWN_REPLICATOR_SECONDS = 30; + + // logger + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpRequestReplicatorImpl.class)); + + // final members + private final Client client; // the client to use for issuing requests + private final int numThreads; // number of threads to use for request replication + private final int connectionTimeoutMs; // connection timeout per node request + private final int readTimeoutMs; // read timeout per node request + + // members + private ExecutorService executorService; + private int shutdownReplicatorSeconds = DEFAULT_SHUTDOWN_REPLICATOR_SECONDS; + + // guarded by synchronized method access in support of multithreaded replication + private String nodeProtocolScheme = null; + + /** + * Creates an instance. The connection timeout and read timeout will be + * infinite. + * + * @param numThreads the number of threads to use when parallelizing + * requests + * @param client a client for making requests + */ + public HttpRequestReplicatorImpl(final int numThreads, final Client client) { + this(numThreads, client, "0 sec", "0 sec"); + } + + /** + * Creates an instance. + * + * @param numThreads the number of threads to use when parallelizing + * requests + * @param client a client for making requests + * @param connectionTimeoutMs the connection timeout specified in + * milliseconds + * @param readTimeoutMs the read timeout specified in milliseconds + */ + public HttpRequestReplicatorImpl(final int numThreads, final Client client, final String connectionTimeout, final String readTimeout) { + + if (numThreads <= 0) { + throw new IllegalArgumentException("The number of threads must be greater than zero."); + } else if (client == null) { + throw new IllegalArgumentException("Client may not be null."); + } + + this.numThreads = numThreads; + this.client = client; + this.connectionTimeoutMs = (int) FormatUtils.getTimeDuration(connectionTimeout, TimeUnit.MILLISECONDS); + this.readTimeoutMs = (int) FormatUtils.getTimeDuration(readTimeout, TimeUnit.MILLISECONDS); + + client.getProperties().put(ClientConfig.PROPERTY_CONNECT_TIMEOUT, connectionTimeoutMs); + client.getProperties().put(ClientConfig.PROPERTY_READ_TIMEOUT, readTimeoutMs); + client.getProperties().put(ClientConfig.PROPERTY_FOLLOW_REDIRECTS, Boolean.TRUE); + } + + @Override + public void start() { + if (isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + executorService = Executors.newFixedThreadPool(numThreads); + } + + @Override + public boolean isRunning() { + return executorService != null && !executorService.isShutdown(); + } + + @Override + public void stop() { + + if (!isRunning()) { + throw new IllegalStateException("Instance is already stopped."); + } + + // shutdown executor service + try { + if (getShutdownReplicatorSeconds() <= 0) { + executorService.shutdownNow(); + } else { + executorService.shutdown(); + } + executorService.awaitTermination(getShutdownReplicatorSeconds(), TimeUnit.SECONDS); + } catch (final InterruptedException ex) { + Thread.currentThread().interrupt(); + } finally { + if (executorService.isTerminated()) { + logger.info("HTTP Request Replicator has been terminated successfully."); + } else { + logger.warn("HTTP Request Replicator has not terminated properly. There exists an uninterruptable thread that will take an indeterminate amount of time to stop."); + } + } + } + + /** + * Sets the protocol scheme to use when issuing requests to nodes. + * + * @param nodeProtocolScheme the scheme. Valid values are "http", "https", + * or null. If null is specified, then the scheme of the originating request + * is used when replicating that request. + */ + public synchronized void setNodeProtocolScheme(final String nodeProtocolScheme) { + if (StringUtils.isNotBlank(nodeProtocolScheme)) { + if (!"http".equalsIgnoreCase(nodeProtocolScheme) && !"https".equalsIgnoreCase(nodeProtocolScheme)) { + throw new IllegalArgumentException("Node Protocol Scheme must be either HTTP or HTTPS"); + } + } + this.nodeProtocolScheme = nodeProtocolScheme; + } + + public synchronized String getNodeProtocolScheme() { + return nodeProtocolScheme; + } + + private synchronized String getNodeProtocolScheme(final URI uri) { + // if we are not configured to use a protocol scheme, then use the uri's scheme + if (StringUtils.isBlank(nodeProtocolScheme)) { + return uri.getScheme(); + } + return nodeProtocolScheme; + } + + public int getConnectionTimeoutMs() { + return connectionTimeoutMs; + } + + public int getReadTimeoutMs() { + return readTimeoutMs; + } + + public int getShutdownReplicatorSeconds() { + return shutdownReplicatorSeconds; + } + + public void setShutdownReplicatorSeconds(int shutdownReplicatorSeconds) { + this.shutdownReplicatorSeconds = shutdownReplicatorSeconds; + } + + @Override + public Set replicate(final Set nodeIds, final String method, + final URI uri, final Map> parameters, final Map headers) + throws UriConstructionException { + if (nodeIds == null) { + throw new IllegalArgumentException("Node IDs may not be null."); + } else if (method == null) { + throw new IllegalArgumentException("HTTP method may not be null."); + } else if (uri == null) { + throw new IllegalArgumentException("URI may not be null."); + } else if (parameters == null) { + throw new IllegalArgumentException("Parameters may not be null."); + } else if (headers == null) { + throw new IllegalArgumentException("HTTP headers map may not be null."); + } + return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), parameters, /* entity */ null, headers); + } + + @Override + public Set replicate(final Set nodeIds, final String method, final URI uri, + final Object entity, final Map headers) throws UriConstructionException { + if (nodeIds == null) { + throw new IllegalArgumentException("Node IDs may not be null."); + } else if (method == null) { + throw new IllegalArgumentException("HTTP method may not be null."); + } else if (method.equalsIgnoreCase(HttpMethod.DELETE) || method.equalsIgnoreCase(HttpMethod.GET) || method.equalsIgnoreCase(HttpMethod.HEAD) || method.equalsIgnoreCase(HttpMethod.OPTIONS)) { + throw new IllegalArgumentException("HTTP (DELETE | GET | HEAD | OPTIONS) requests cannot have a body containing an entity."); + } else if (uri == null) { + throw new IllegalArgumentException("URI may not be null."); + } else if (entity == null) { + throw new IllegalArgumentException("Entity may not be null."); + } else if (headers == null) { + throw new IllegalArgumentException("HTTP headers map may not be null."); + } + return replicateHelper(nodeIds, method, getNodeProtocolScheme(uri), uri.getPath(), /* parameters */ null, entity, headers); + } + + private Set replicateHelper(final Set nodeIds, final String method, final String scheme, + final String path, final Map> parameters, final Object entity, final Map headers) + throws UriConstructionException { + + if (nodeIds.isEmpty()) { + return new HashSet<>(); // return quickly for trivial case + } + + final CompletionService completionService = new ExecutorCompletionService<>(executorService); + + // keeps track of future requests so that failed requests can be tied back to the failing node + final Collection futureNodeHttpRequests = new ArrayList<>(); + + // construct the URIs for the nodes + final Map uriMap = new HashMap<>(); + try { + for (final NodeIdentifier nodeId : nodeIds) { + final URI nodeUri = new URI(scheme, null, nodeId.getApiAddress(), nodeId.getApiPort(), path, /* query */ null, /* fragment */ null); + uriMap.put(nodeId, nodeUri); + } + } catch (final URISyntaxException use) { + throw new UriConstructionException(use); + } + + // submit the requests to the nodes + final String requestId = UUID.randomUUID().toString(); + headers.put(WebClusterManager.REQUEST_ID_HEADER, requestId); + for (final Map.Entry entry : uriMap.entrySet()) { + final NodeIdentifier nodeId = entry.getKey(); + final URI nodeUri = entry.getValue(); + final NodeHttpRequestCallable callable = (entity == null) + ? new NodeHttpRequestCallable(nodeId, method, nodeUri, parameters, headers) + : new NodeHttpRequestCallable(nodeId, method, nodeUri, entity, headers); + futureNodeHttpRequests.add(new NodeHttpRequestFutureWrapper(nodeId, method, nodeUri, completionService.submit(callable))); + } + + // get the node responses + final Set result = new HashSet<>(); + for (int i = 0; i < nodeIds.size(); i++) { + + // keeps track of the original request information in case we receive an exception + NodeHttpRequestFutureWrapper futureNodeHttpRequest = null; + try { + + // get the future resource response for the node + final Future futureNodeResourceResponse = completionService.take(); + + // find the original request by comparing the submitted future with the future returned by the completion service + for (final NodeHttpRequestFutureWrapper futureNodeHttpRequestElem : futureNodeHttpRequests) { + if (futureNodeHttpRequestElem.getFuture() == futureNodeResourceResponse) { + futureNodeHttpRequest = futureNodeHttpRequestElem; + } + } + + // try to retrieve the node response and add to result + final NodeResponse nodeResponse = futureNodeResourceResponse.get(); + result.add(nodeResponse); + + } catch (final InterruptedException | ExecutionException ex) { + + logger.warn("Node request for " + futureNodeHttpRequest.getNodeId() + " encountered exception: " + ex, ex); + + // create node response with the thrown exception and add to result + final NodeResponse nodeResponse = new NodeResponse( + futureNodeHttpRequest.getNodeId(), futureNodeHttpRequest.getHttpMethod(), futureNodeHttpRequest.getRequestUri(), ex); + result.add(nodeResponse); + + } + } + + if (logger.isDebugEnabled()) { + NodeResponse min = null; + NodeResponse max = null; + long nanosSum = 0L; + int nanosAdded = 0; + + for (final NodeResponse response : result) { + final long requestNanos = response.getRequestDuration(TimeUnit.NANOSECONDS); + final long minNanos = (min == null) ? -1 : min.getRequestDuration(TimeUnit.NANOSECONDS); + final long maxNanos = (max == null) ? -1 : max.getRequestDuration(TimeUnit.NANOSECONDS); + + if (requestNanos < minNanos || minNanos < 0L) { + min = response; + } + + if (requestNanos > maxNanos || maxNanos < 0L) { + max = response; + } + + if (requestNanos >= 0L) { + nanosSum += requestNanos; + nanosAdded++; + } + } + + final StringBuilder sb = new StringBuilder(); + sb.append("Node Responses for ").append(method).append(" ").append(path).append(" (Request ID ").append(requestId).append("):\n"); + for (final NodeResponse response : result) { + sb.append(response).append("\n"); + } + + final long averageNanos = (nanosAdded == 0) ? -1L : nanosSum / nanosAdded; + final long averageMillis = (averageNanos < 0) ? averageNanos : TimeUnit.MILLISECONDS.convert(averageNanos, TimeUnit.NANOSECONDS); + logger.debug("For {} {} (Request ID {}), minimum response time = {}, max = {}, average = {} ms", + method, path, requestId, min, max, averageMillis); + logger.debug(sb.toString()); + } + + return result; + } + + /** + * Wraps a future node response with info from originating request. This + * coupling allows for futures that encountered exceptions to be linked back + * to the failing node and better reported. + */ + private class NodeHttpRequestFutureWrapper { + + private final NodeIdentifier nodeId; + + private final String httpMethod; + + private final URI requestUri; + + private final Future future; + + public NodeHttpRequestFutureWrapper(final NodeIdentifier nodeId, final String httpMethod, + final URI requestUri, final Future future) { + if (nodeId == null) { + throw new IllegalArgumentException("Node ID may not be null."); + } else if (StringUtils.isBlank(httpMethod)) { + throw new IllegalArgumentException("Http method may not be null or empty."); + } else if (requestUri == null) { + throw new IllegalArgumentException("Request URI may not be null."); + } else if (future == null) { + throw new IllegalArgumentException("Future may not be null."); + } + this.nodeId = nodeId; + this.httpMethod = httpMethod; + this.requestUri = requestUri; + this.future = future; + } + + public NodeIdentifier getNodeId() { + return nodeId; + } + + public String getHttpMethod() { + return httpMethod; + } + + public URI getRequestUri() { + return requestUri; + } + + public Future getFuture() { + return future; + } + } + + /** + * A Callable for making an HTTP request to a single node and returning its + * response. + */ + private class NodeHttpRequestCallable implements Callable { + + private final NodeIdentifier nodeId; + private final String method; + private final URI uri; + private final Object entity; + private final Map> parameters = new HashMap<>(); + private final Map headers = new HashMap<>(); + + private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method, + final URI uri, final Object entity, final Map headers) { + this.nodeId = nodeId; + this.method = method; + this.uri = uri; + this.entity = entity; + this.headers.putAll(headers); + } + + private NodeHttpRequestCallable(final NodeIdentifier nodeId, final String method, + final URI uri, final Map> parameters, final Map headers) { + this.nodeId = nodeId; + this.method = method; + this.uri = uri; + this.entity = null; + this.parameters.putAll(parameters); + this.headers.putAll(headers); + } + + @Override + public NodeResponse call() { + + try { + // create and send the request + final WebResource.Builder resourceBuilder = getResourceBuilder(); + final String requestId = headers.get("x-nifi-request-id"); + + final long startNanos = System.nanoTime(); + final ClientResponse clientResponse; + if (HttpMethod.DELETE.equalsIgnoreCase(method)) { + clientResponse = resourceBuilder.delete(ClientResponse.class); + } else if (HttpMethod.GET.equalsIgnoreCase(method)) { + clientResponse = resourceBuilder.get(ClientResponse.class); + } else if (HttpMethod.HEAD.equalsIgnoreCase(method)) { + clientResponse = resourceBuilder.head(); + } else if (HttpMethod.OPTIONS.equalsIgnoreCase(method)) { + clientResponse = resourceBuilder.options(ClientResponse.class); + } else if (HttpMethod.POST.equalsIgnoreCase(method)) { + clientResponse = resourceBuilder.post(ClientResponse.class); + } else if (HttpMethod.PUT.equalsIgnoreCase(method)) { + clientResponse = resourceBuilder.put(ClientResponse.class); + } else { + throw new IllegalArgumentException("HTTP Method '" + method + "' not supported for request replication."); + } + + // create and return the response + return new NodeResponse(nodeId, method, uri, clientResponse, System.nanoTime() - startNanos, requestId); + + } catch (final UniformInterfaceException | IllegalArgumentException t) { + return new NodeResponse(nodeId, method, uri, t); + } + + } + + private WebResource.Builder getResourceBuilder() { + + // convert parameters to a more convenient data structure + final MultivaluedMap map = new MultivaluedMapImpl(); + map.putAll(parameters); + + // create the resource + WebResource resource = client.resource(uri); + + if (WebClusterManager.isResponseInterpreted(uri, method)) { + resource.addFilter(new GZIPContentEncodingFilter(false)); + } + + // set the parameters as either query parameters or as request body + final WebResource.Builder builder; + if (HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.HEAD.equalsIgnoreCase(method) || HttpMethod.GET.equalsIgnoreCase(method) || HttpMethod.OPTIONS.equalsIgnoreCase(method)) { + resource = resource.queryParams(map); + builder = resource.getRequestBuilder(); + } else { + if (entity == null) { + builder = resource.entity(map); + } else { + builder = resource.entity(entity); + } + } + + // set headers + boolean foundContentType = false; + for (final Map.Entry entry : headers.entrySet()) { + builder.header(entry.getKey(), entry.getValue()); + if (entry.getKey().equalsIgnoreCase("content-type")) { + foundContentType = true; + } + } + + // set default content type + if (!foundContentType) { + // set default content type + builder.type(MediaType.APPLICATION_FORM_URLENCODED); + } + + return builder; + } + + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java new file mode 100644 index 0000000000..afade7edce --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImpl.java @@ -0,0 +1,85 @@ +/* + * 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.cluster.manager.impl; + +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.nifi.cluster.manager.HttpResponseMapper; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.node.Node; +import org.apache.nifi.cluster.node.Node.Status; +import org.apache.nifi.logging.NiFiLog; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Determines the status of nodes based on their HTTP response codes. + * + * The algorithm is as follows. + * + * If any HTTP responses were 2XX, then disconnect non-2XX responses. This is + * because 2XX may have changed a node's flow. + * + * If no 2XX responses were received, then the node's flow has not changed. + * Instead of disconnecting everything, we only disconnect the nodes with + * internal errors, i.e., 5XX responses. + * + * @author unattributed + */ +public class HttpResponseMapperImpl implements HttpResponseMapper { + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(HttpResponseMapperImpl.class)); + + @Override + public Map map(final URI requestURI, final Set nodeResponses) { + + final Map result = new HashMap<>(); + + // check if any responses were 2XX + boolean found2xx = false; + for (final NodeResponse nodeResponse : nodeResponses) { + if (nodeResponse.is2xx()) { + found2xx = true; + break; + } + } + + // determine the status of each node + for (final NodeResponse nodeResponse : nodeResponses) { + + final Node.Status status; + if (found2xx) { + // disconnect nodes with non-2XX responses + status = nodeResponse.is2xx() + ? Node.Status.CONNECTED + : Node.Status.DISCONNECTED; + } else { + // disconnect nodes with 5XX responses or exception + status = nodeResponse.is5xx() + ? Node.Status.DISCONNECTED + : Node.Status.CONNECTED; + } + + result.put(nodeResponse, status); + } + + return result; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java new file mode 100644 index 0000000000..3defea75e0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/manager/impl/WebClusterManager.java @@ -0,0 +1,3616 @@ +/* + * 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.cluster.manager.impl; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.io.Serializable; +import java.net.URI; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.ListIterator; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.Timer; +import java.util.TimerTask; +import java.util.TreeMap; +import java.util.UUID; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.regex.Pattern; + +import javax.net.ssl.SSLContext; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.WebApplicationException; +import javax.ws.rs.core.StreamingOutput; +import javax.xml.XMLConstants; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; +import javax.xml.transform.dom.DOMSource; +import javax.xml.validation.Schema; +import javax.xml.validation.SchemaFactory; +import javax.xml.validation.Validator; + +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.cluster.BulletinsPayload; +import org.apache.nifi.cluster.ClusterNodeInformation; +import org.apache.nifi.cluster.HeartbeatPayload; +import org.apache.nifi.cluster.NodeInformation; +import org.apache.nifi.cluster.context.ClusterContext; +import org.apache.nifi.cluster.context.ClusterContextImpl; +import org.apache.nifi.cluster.event.Event; +import org.apache.nifi.cluster.event.EventManager; +import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; +import org.apache.nifi.cluster.flow.ClusterDataFlow; +import org.apache.nifi.cluster.flow.DaoException; +import org.apache.nifi.cluster.flow.DataFlowManagementService; +import org.apache.nifi.cluster.flow.PersistedFlowState; +import org.apache.nifi.cluster.manager.HttpClusterManager; +import org.apache.nifi.cluster.manager.HttpRequestReplicator; +import org.apache.nifi.cluster.manager.HttpResponseMapper; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.manager.exception.ConnectingNodeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.DisconnectedNodeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.IllegalClusterStateException; +import org.apache.nifi.cluster.manager.exception.IllegalNodeDeletionException; +import org.apache.nifi.cluster.manager.exception.IllegalNodeDisconnectionException; +import org.apache.nifi.cluster.manager.exception.IllegalNodeReconnectionException; +import org.apache.nifi.cluster.manager.exception.IneligiblePrimaryNodeException; +import org.apache.nifi.cluster.manager.exception.NoConnectedNodesException; +import org.apache.nifi.cluster.manager.exception.NoResponseFromNodesException; +import org.apache.nifi.cluster.manager.exception.NodeDisconnectionException; +import org.apache.nifi.cluster.manager.exception.NodeReconnectionException; +import org.apache.nifi.cluster.manager.exception.PrimaryRoleAssignmentException; +import org.apache.nifi.cluster.manager.exception.SafeModeMutableRequestException; +import org.apache.nifi.cluster.manager.exception.UnknownNodeException; +import org.apache.nifi.cluster.manager.exception.UriConstructionException; +import org.apache.nifi.cluster.node.Node; +import org.apache.nifi.cluster.node.Node.Status; +import org.apache.nifi.cluster.protocol.ConnectionRequest; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.NodeBulletins; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener; +import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.DisconnectMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.NodeBulletinsMessage; +import org.apache.nifi.cluster.protocol.message.PrimaryRoleAssignmentMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage.MessageType; +import org.apache.nifi.cluster.protocol.message.ReconnectionRequestMessage; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.controller.Heartbeater; +import org.apache.nifi.controller.ReportingTaskNode; +import org.apache.nifi.controller.ValidationContextFactory; +import org.apache.nifi.controller.reporting.ClusteredReportingTaskNode; +import org.apache.nifi.controller.reporting.ReportingTaskInstantiationException; +import org.apache.nifi.controller.reporting.StandardReportingInitializationContext; +import org.apache.nifi.controller.scheduling.StandardProcessScheduler; +import org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.controller.service.StandardControllerServiceProvider; +import org.apache.nifi.controller.status.ProcessGroupStatus; +import org.apache.nifi.controller.status.RemoteProcessGroupStatus; +import org.apache.nifi.controller.status.history.ComponentStatusRepository; +import org.apache.nifi.controller.status.history.MetricDescriptor; +import org.apache.nifi.controller.status.history.StatusHistory; +import org.apache.nifi.controller.status.history.StatusHistoryUtil; +import org.apache.nifi.controller.status.history.StatusSnapshot; +import org.apache.nifi.diagnostics.GarbageCollection; +import org.apache.nifi.diagnostics.StorageUsage; +import org.apache.nifi.diagnostics.SystemDiagnostics; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.engine.FlowEngine; +import org.apache.nifi.events.BulletinFactory; +import org.apache.nifi.events.VolatileBulletinRepository; +import org.apache.nifi.framework.security.util.SslContextFactory; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.logging.NiFiLog; +import org.apache.nifi.nar.ExtensionManager; +import org.apache.nifi.nar.NarCloseable; +import org.apache.nifi.nar.NarThreadContextClassLoader; +import org.apache.nifi.processor.StandardValidationContextFactory; +import org.apache.nifi.remote.RemoteResourceManager; +import org.apache.nifi.remote.RemoteSiteListener; +import org.apache.nifi.remote.SocketRemoteSiteListener; +import org.apache.nifi.remote.protocol.socket.ClusterManagerServerProtocol; +import org.apache.nifi.reporting.Bulletin; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.reporting.ReportingInitializationContext; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.reporting.Severity; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.DomUtils; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; +import org.apache.nifi.web.Revision; +import org.apache.nifi.web.api.dto.FlowSnippetDTO; +import org.apache.nifi.web.api.dto.NodeDTO; +import org.apache.nifi.web.api.dto.ProcessGroupDTO; +import org.apache.nifi.web.api.dto.ProcessorDTO; +import org.apache.nifi.web.api.dto.RemoteProcessGroupContentsDTO; +import org.apache.nifi.web.api.dto.RemoteProcessGroupDTO; +import org.apache.nifi.web.api.dto.RemoteProcessGroupPortDTO; +import org.apache.nifi.web.api.dto.provenance.ProvenanceDTO; +import org.apache.nifi.web.api.dto.provenance.ProvenanceEventDTO; +import org.apache.nifi.web.api.dto.provenance.ProvenanceRequestDTO; +import org.apache.nifi.web.api.dto.provenance.ProvenanceResultsDTO; +import org.apache.nifi.web.api.dto.status.ClusterStatusHistoryDTO; +import org.apache.nifi.web.api.dto.status.NodeStatusHistoryDTO; +import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; +import org.apache.nifi.web.api.dto.status.StatusSnapshotDTO; +import org.apache.nifi.web.api.entity.FlowSnippetEntity; +import org.apache.nifi.web.api.entity.ProcessGroupEntity; +import org.apache.nifi.web.api.entity.ProcessorEntity; +import org.apache.nifi.web.api.entity.ProcessorsEntity; +import org.apache.nifi.web.api.entity.ProvenanceEntity; +import org.apache.nifi.web.api.entity.ProvenanceEventEntity; +import org.apache.nifi.web.api.entity.RemoteProcessGroupEntity; +import org.apache.nifi.web.api.entity.RemoteProcessGroupsEntity; +import org.apache.nifi.web.util.WebUtils; + +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.DOMException; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.NodeList; +import org.xml.sax.SAXException; +import org.xml.sax.SAXParseException; + +import com.sun.jersey.api.client.ClientResponse; + +import org.apache.nifi.cluster.protocol.message.ReconnectionFailureMessage; + +/** + * Provides a cluster manager implementation. The manager federates incoming + * HTTP client requests to the nodes' external API using the HTTP protocol. The + * manager also communicates with nodes using the nodes' internal socket + * protocol. + * + * The manager's socket address may broadcasted using multicast if a + * MulticastServiceBroadcaster instance is set on this instance. The manager + * instance must be started after setting the broadcaster. + * + * The manager may be configured with an EventManager for recording noteworthy + * lifecycle events (e.g., first heartbeat received, node status change). + * + * The start() and stop() methods must be called to initialize and stop the + * instance. + * + * @author unattributed + */ +public class WebClusterManager implements HttpClusterManager, ProtocolHandler, ControllerServiceProvider { + + public static final String ROOT_GROUP_ID_ALIAS = "root"; + public static final String BULLETIN_CATEGORY = "Clustering"; + + private static final Logger logger = new NiFiLog(LoggerFactory.getLogger(WebClusterManager.class)); + private static final Logger heartbeatLogger = new NiFiLog(LoggerFactory.getLogger("org.apache.nifi.cluster.heartbeat")); + + /** + * The HTTP header to store a cluster context. An example of what may be + * stored in the context is a node's auditable actions in response to a + * cluster request. The cluster context is serialized using Java's + * serialization mechanism and hex encoded. + */ + public static final String CLUSTER_CONTEXT_HTTP_HEADER = "X-ClusterContext"; + + /** + * HTTP Header that stores a unique ID for each request that is replicated + * to the nodes. This is used for logging purposes so that request + * information, such as timing, can be correlated between the NCM and the + * nodes + */ + public static final String REQUEST_ID_HEADER = "X-RequestID"; + + /** + * The HTTP header that the NCM specifies to ask a node if they are able to + * process a given request. The value is always 150-NodeContinue. The node + * will respond with 150 CONTINUE if it is able to process the request, 417 + * EXPECTATION_FAILED otherwise. + */ + public static final String NCM_EXPECTS_HTTP_HEADER = "X-NcmExpects"; + public static final int NODE_CONTINUE_STATUS_CODE = 150; + + /** + * The HTTP header that the NCM specifies to indicate that a node should + * invalidate the specified user group. This is done to ensure that user + * cache is not stale when an administrator modifies a group through the UI. + */ + public static final String CLUSTER_INVALIDATE_USER_GROUP_HEADER = "X-ClusterInvalidateUserGroup"; + + /** + * The HTTP header that the NCM specifies to indicate that a node should + * invalidate the specified user. This is done to ensure that user cache is + * not stale when an administrator modifies a user through the UI. + */ + public static final String CLUSTER_INVALIDATE_USER_HEADER = "X-ClusterInvalidateUser"; + + /** + * The default number of seconds to respond to a connecting node if the + * manager cannot provide it with a current data flow. + */ + private static final int DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS = 5; + + public static final String DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION = "org.apache.nifi.controller.status.history.VolatileComponentStatusRepository"; + + public static final Pattern PROCESSORS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors"); + public static final Pattern PROCESSOR_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/processors/[a-f0-9\\-]{36}"); + + public static final Pattern REMOTE_PROCESS_GROUPS_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups"); + public static final Pattern REMOTE_PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/remote-process-groups/[a-f0-9\\-]{36}"); + + public static final Pattern PROCESS_GROUP_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))"); + public static final Pattern TEMPLATE_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/template-instance"); + public static final Pattern FLOW_SNIPPET_INSTANCE_URI_PATTERN = Pattern.compile("/nifi-api/controller/process-groups/(?:(?:root)|(?:[a-f0-9\\-]{36}))/snippet-instance"); + + public static final String PROVENANCE_URI = "/nifi-api/controller/provenance"; + public static final Pattern PROVENANCE_QUERY_URI = Pattern.compile("/nifi-api/controller/provenance/[a-f0-9\\-]{36}"); + public static final Pattern PROVENANCE_EVENT_URI = Pattern.compile("/nifi-api/controller/provenance/events/[0-9]+"); + + private final NiFiProperties properties; + private final HttpRequestReplicator httpRequestReplicator; + private final HttpResponseMapper httpResponseMapper; + private final DataFlowManagementService dataFlowManagementService; + private final ClusterManagerProtocolSenderListener senderListener; + private final StringEncryptor encryptor; + private final Queue pendingHeartbeats = new ConcurrentLinkedQueue<>(); + private final ReentrantReadWriteLock resourceRWLock = new ReentrantReadWriteLock(); + private final ClusterManagerLock readLock = new ClusterManagerLock(resourceRWLock.readLock(), "Read"); + private final ClusterManagerLock writeLock = new ClusterManagerLock(resourceRWLock.writeLock(), "Write"); + + private final Set nodes = new HashSet<>(); + private final Set reportingTasks = new HashSet<>(); + + // null means the dataflow should be read from disk + private StandardDataFlow cachedDataFlow = null; + private NodeIdentifier primaryNodeId = null; + private Revision revision = new Revision(0L, ""); + private Timer heartbeatMonitor; + private Timer heartbeatProcessor; + private volatile ClusterServicesBroadcaster servicesBroadcaster = null; + private volatile EventManager eventManager = null; + private volatile ClusterNodeFirewall clusterFirewall = null; + private volatile AuditService auditService = null; + private volatile ControllerServiceProvider controllerServiceProvider = null; + + private final RemoteSiteListener remoteSiteListener; + private final Integer remoteInputPort; + private final Boolean remoteCommsSecure; + private final BulletinRepository bulletinRepository; + private final String instanceId; + private final FlowEngine reportingTaskEngine; + private final Map componentMetricsRepositoryMap = new HashMap<>(); + private final StandardProcessScheduler processScheduler; + private final long componentStatusSnapshotMillis; + + public WebClusterManager(final HttpRequestReplicator httpRequestReplicator, final HttpResponseMapper httpResponseMapper, + final DataFlowManagementService dataFlowManagementService, final ClusterManagerProtocolSenderListener senderListener, + final NiFiProperties properties, final StringEncryptor encryptor) { + + if (httpRequestReplicator == null) { + throw new IllegalArgumentException("HttpRequestReplicator may not be null."); + } else if (httpResponseMapper == null) { + throw new IllegalArgumentException("HttpResponseMapper may not be null."); + } else if (dataFlowManagementService == null) { + throw new IllegalArgumentException("DataFlowManagementService may not be null."); + } else if (senderListener == null) { + throw new IllegalArgumentException("ClusterManagerProtocolSenderListener may not be null."); + } else if (properties == null) { + throw new IllegalArgumentException("NiFiProperties may not be null."); + } + + // Ensure that our encryptor/decryptor is properly initialized + this.httpRequestReplicator = httpRequestReplicator; + this.httpResponseMapper = httpResponseMapper; + this.dataFlowManagementService = dataFlowManagementService; + this.properties = properties; + this.controllerServiceProvider = new StandardControllerServiceProvider(); + this.bulletinRepository = new VolatileBulletinRepository(); + this.instanceId = UUID.randomUUID().toString(); + this.senderListener = senderListener; + this.encryptor = encryptor; + senderListener.addHandler(this); + senderListener.setBulletinRepository(bulletinRepository); + + final String snapshotFrequency = properties.getProperty(NiFiProperties.COMPONENT_STATUS_SNAPSHOT_FREQUENCY, NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY); + long snapshotMillis; + try { + snapshotMillis = FormatUtils.getTimeDuration(snapshotFrequency, TimeUnit.MILLISECONDS); + } catch (final Exception e) { + snapshotMillis = FormatUtils.getTimeDuration(NiFiProperties.DEFAULT_COMPONENT_STATUS_SNAPSHOT_FREQUENCY, TimeUnit.MILLISECONDS); + } + componentStatusSnapshotMillis = snapshotMillis; + + remoteInputPort = properties.getRemoteInputPort(); + if (remoteInputPort == null) { + remoteSiteListener = null; + remoteCommsSecure = null; + } else { + // Register the ClusterManagerServerProtocol as the appropriate resource for site-to-site Server Protocol + RemoteResourceManager.setServerProtocolImplementation(ClusterManagerServerProtocol.RESOURCE_NAME, ClusterManagerServerProtocol.class); + remoteCommsSecure = properties.isSiteToSiteSecure(); + if (remoteCommsSecure) { + final SSLContext sslContext = SslContextFactory.createSslContext(properties, false); + + if (sslContext == null) { + throw new IllegalStateException("NiFi Configured to allow Secure Site-to-Site communications but the Keystore/Truststore properties are not configured"); + } + + remoteSiteListener = new SocketRemoteSiteListener(remoteInputPort.intValue(), sslContext, this); + } else { + remoteSiteListener = new SocketRemoteSiteListener(remoteInputPort.intValue(), null, this); + } + } + + reportingTaskEngine = new FlowEngine(8, "Reporting Task Thread"); + + processScheduler = new StandardProcessScheduler(new Heartbeater() { + @Override + public void heartbeat() { + } + }, this, encryptor); + processScheduler.setSchedulingAgent(SchedulingStrategy.TIMER_DRIVEN, new TimerDrivenSchedulingAgent(null, reportingTaskEngine, null, encryptor)); + processScheduler.setMaxThreadCount(SchedulingStrategy.TIMER_DRIVEN, 10); + processScheduler.setMaxThreadCount(SchedulingStrategy.CRON_DRIVEN, 10); + } + + public void start() throws IOException { + writeLock.lock(); + try { + + if (isRunning()) { + throw new IllegalStateException("Instance is already started."); + } + + try { + // setup heartbeat monitoring + heartbeatMonitor = new Timer("Heartbeat Monitor", /* is daemon */ true); + heartbeatMonitor.scheduleAtFixedRate(new HeartbeatMonitoringTimerTask(), 0, getHeartbeatMonitoringIntervalSeconds() * 1000); + + heartbeatProcessor = new Timer("Process Pending Heartbeats", true); + final int processPendingHeartbeatDelay = 1000 * Math.max(1, getClusterProtocolHeartbeatSeconds() / 2); + heartbeatProcessor.schedule(new ProcessPendingHeartbeatsTask(), processPendingHeartbeatDelay, processPendingHeartbeatDelay); + + // start request replication service + httpRequestReplicator.start(); + + // start protocol service + senderListener.start(); + + // start flow management service + dataFlowManagementService.start(); + + if (remoteSiteListener != null) { + remoteSiteListener.start(); + } + + // load flow + if (dataFlowManagementService.isFlowCurrent()) { + final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); + cachedDataFlow = clusterDataFlow.getDataFlow(); + primaryNodeId = clusterDataFlow.getPrimaryNodeId(); + } else { + throw new IOException("Flow is not current."); + } + + // start multicast broadcasting service, if configured + if (servicesBroadcaster != null) { + servicesBroadcaster.start(); + } + + // start in safe mode + executeSafeModeTask(); + + // Load and start running Reporting Tasks + final File taskFile = new File(properties.getProperty(NiFiProperties.TASK_CONFIGURATION_FILE)); + reportingTasks.addAll(loadReportingTasks(taskFile)); + } catch (final IOException ioe) { + logger.warn("Failed to initialize cluster services due to: " + ioe, ioe); + stop(); + throw ioe; + } + + } finally { + writeLock.unlock("START"); + } + } + + public void stop() throws IOException { + writeLock.lock(); + try { + + // returns true if any service is running + if (isRunning() == false) { + throw new IllegalArgumentException("Instance is already stopped."); + } + + boolean encounteredException = false; + + // stop the heartbeat monitoring + if (isHeartbeatMonitorRunning()) { + heartbeatMonitor.cancel(); + heartbeatMonitor = null; + } + + if (heartbeatProcessor != null) { + heartbeatProcessor.cancel(); + heartbeatProcessor = null; + } + + // stop the HTTP request replicator service + if (httpRequestReplicator.isRunning()) { + httpRequestReplicator.stop(); + } + + // stop the flow management service + if (dataFlowManagementService.isRunning()) { + dataFlowManagementService.stop(); + } + + if (remoteSiteListener != null) { + remoteSiteListener.stop(); + } + + // stop the protocol listener service + if (senderListener.isRunning()) { + try { + senderListener.stop(); + } catch (final IOException ioe) { + encounteredException = true; + logger.warn("Failed to shutdown protocol service due to: " + ioe, ioe); + } + } + + // stop the service broadcaster + if (isBroadcasting()) { + servicesBroadcaster.stop(); + } + + if (encounteredException) { + throw new IOException("Failed to shutdown Cluster Manager because one or more cluster services failed to shutdown. Check the logs for details."); + } + + } finally { + writeLock.unlock("STOP"); + } + } + + public boolean isRunning() { + readLock.lock(); + try { + return isHeartbeatMonitorRunning() + || httpRequestReplicator.isRunning() + || senderListener.isRunning() + || dataFlowManagementService.isRunning() + || isBroadcasting(); + } finally { + readLock.unlock("isRunning"); + } + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return MessageType.CONNECTION_REQUEST == msg.getType() + || MessageType.HEARTBEAT == msg.getType() + || MessageType.CONTROLLER_STARTUP_FAILURE == msg.getType() + || MessageType.BULLETINS == msg.getType() + || MessageType.RECONNECTION_FAILURE == msg.getType(); + } + + @Override + public ProtocolMessage handle(final ProtocolMessage protocolMessage) throws ProtocolException { + switch (protocolMessage.getType()) { + case CONNECTION_REQUEST: + return handleConnectionRequest((ConnectionRequestMessage) protocolMessage); + case HEARTBEAT: + final HeartbeatMessage heartbeatMessage = (HeartbeatMessage) protocolMessage; + + final Heartbeat original = heartbeatMessage.getHeartbeat(); + final NodeIdentifier originalNodeId = original.getNodeIdentifier(); + final Heartbeat heartbeatWithDn = new Heartbeat(addRequestorDn(originalNodeId, heartbeatMessage.getRequestorDN()), original.isPrimary(), original.isConnected(), original.getPayload()); + + handleHeartbeat(heartbeatWithDn); + return null; + case CONTROLLER_STARTUP_FAILURE: + new Thread(new Runnable() { + @Override + public void run() { + handleControllerStartupFailure((ControllerStartupFailureMessage) protocolMessage); + } + }, "Handle Controller Startup Failure Message from " + ((ControllerStartupFailureMessage) protocolMessage).getNodeId()).start(); + return null; + case RECONNECTION_FAILURE: + new Thread(new Runnable() { + @Override + public void run() { + handleReconnectionFailure((ReconnectionFailureMessage) protocolMessage); + } + }, "Handle Reconnection Failure Message from " + ((ReconnectionFailureMessage) protocolMessage).getNodeId()).start(); + return null; + case BULLETINS: + final NodeBulletinsMessage bulletinsMessage = (NodeBulletinsMessage) protocolMessage; + handleBulletins(bulletinsMessage.getBulletins()); + return null; + default: + throw new ProtocolException("No handler defined for message type: " + protocolMessage.getType()); + } + } + + /** + * Services connection requests. If the data flow management service is + * unable to provide a current copy of the data flow, then the returned + * connection response will indicate the node should try later. Otherwise, + * the connection response will contain the the flow and the node + * identifier. + * + * If this instance is configured with a firewall and the request is + * blocked, then the response will not contain a node identifier. + * + * @param request a connection request + * + * @return a connection response + */ + @Override + public ConnectionResponse requestConnection(final ConnectionRequest request) { + final boolean lockObtained = writeLock.tryLock(3, TimeUnit.SECONDS); + if (!lockObtained) { + // Create try-later response because we are too busy to service the request right now. We do not want + // to wait long because we want Node/NCM comms to be very responsive + final int tryAgainSeconds; + if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) { + tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS; + } else { + tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds(); + } + + // record event + final String msg = "Connection requested from node, but manager was too busy to service request. Instructing node to try again in " + tryAgainSeconds + " seconds."; + addEvent(request.getProposedNodeIdentifier(), msg); + addBulletin(request.getProposedNodeIdentifier(), Severity.INFO, msg); + + // return try later response + return new ConnectionResponse(tryAgainSeconds); + } + + try { + // resolve the proposed node identifier to a valid node identifier + final NodeIdentifier resolvedNodeIdentifier = resolveProposedNodeIdentifier(request.getProposedNodeIdentifier()); + + if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) { + // if the socket address is not listed in the firewall, then return a null response + logger.info("Firewall blocked connection request from node " + resolvedNodeIdentifier); + return ConnectionResponse.createBlockedByFirewallResponse(); + } + + // get a raw reference to the node (if it doesn't exist, node will be null) + Node node = getRawNode(resolvedNodeIdentifier.getId()); + + // create a new node if necessary and set status to connecting + if (node == null) { + node = new Node(resolvedNodeIdentifier, Status.CONNECTING); + addEvent(node.getNodeId(), "Connection requested from new node. Setting status to connecting."); + nodes.add(node); + } else { + node.setStatus(Status.CONNECTING); + addEvent(resolvedNodeIdentifier, "Connection requested from existing node. Setting status to connecting"); + } + + // record the time of the connection request + node.setConnectionRequestedTimestamp(new Date().getTime()); + + // clear out old heartbeat info + node.setHeartbeat(null); + + // try to obtain a current flow + if (dataFlowManagementService.isFlowCurrent()) { + // if a cached copy does not exist, load it from disk + if (cachedDataFlow == null) { + final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); + cachedDataFlow = clusterDataFlow.getDataFlow(); + primaryNodeId = clusterDataFlow.getPrimaryNodeId(); + } + + // determine if this node should be assigned the primary role + final boolean primaryRole; + if (primaryNodeId == null || primaryNodeId.logicallyEquals(node.getNodeId())) { + setPrimaryNodeId(node.getNodeId()); + addEvent(node.getNodeId(), "Setting primary role in connection response."); + primaryRole = true; + } else { + primaryRole = false; + } + + return new ConnectionResponse(node.getNodeId(), cachedDataFlow, primaryRole, remoteInputPort, remoteCommsSecure, instanceId); + } + + /* + * The manager does not have a current copy of the data flow, + * so it will instruct the node to try connecting at a later + * time. Meanwhile, the flow will be locked down from user + * changes because the node is marked as connecting. + */ + + /* + * Create try-later response based on flow retrieval delay to give + * the flow management service a chance to retrieve a curren flow + */ + final int tryAgainSeconds; + if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) { + tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS; + } else { + tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds(); + } + + // record event + addEvent(node.getNodeId(), "Connection requested from node, but manager was unable to obtain current flow. Instructing node to try again in " + tryAgainSeconds + " seconds."); + + // return try later response + return new ConnectionResponse(tryAgainSeconds); + + } finally { + writeLock.unlock("requestConnection"); + } + } + + /** + * Services reconnection requests for a given node. If the node indicates + * reconnection failure, then the node will be set to disconnected and if + * the node has primary role, then the role will be revoked. Otherwise, a + * reconnection request will be sent to the node, initiating the connection + * handshake. + * + * @param nodeId a node identifier + * + * @throws UnknownNodeException if the node does not exist + * @throws IllegalNodeReconnectionException if the node cannot be + * reconnected because the node is not disconnected + * @throws NodeReconnectionException if the reconnection message failed to + * be sent or the cluster could not provide a current data flow for the + * reconnection request + */ + @Override + public void requestReconnection(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeReconnectionException { + Node node = null; + + final boolean primaryRole; + final int tryAgainSeconds; + + writeLock.lock(); + try { + // check if we know about this node and that it is disconnected + node = getRawNode(nodeId); + logger.info("Request was made by {} to reconnect node {} to cluster", userDn, node == null ? nodeId : node); + + if (node == null) { + throw new UnknownNodeException("Node does not exist."); + } else if (Status.DISCONNECTED != node.getStatus()) { + throw new IllegalNodeReconnectionException("Node must be disconnected before it can reconnect."); + } + + // clear out old heartbeat info + node.setHeartbeat(null); + + // get the dataflow to send with the reconnection request + if (!dataFlowManagementService.isFlowCurrent()) { + /* node remains disconnected */ + final String msg = "Reconnection requested for node, but manager was unable to obtain current flow. Setting node to disconnected."; + addEvent(node.getNodeId(), msg); + addBulletin(node, Severity.WARNING, msg); + throw new NodeReconnectionException("Manager was unable to obtain current flow to provide in reconnection request to node. Try again in a few seconds."); + } + + // if a cached copy does not exist, load it from disk + if (cachedDataFlow == null) { + final ClusterDataFlow clusterDataFlow = dataFlowManagementService.loadDataFlow(); + cachedDataFlow = clusterDataFlow.getDataFlow(); + primaryNodeId = clusterDataFlow.getPrimaryNodeId(); + } + + node.setStatus(Status.CONNECTING); + addEvent(node.getNodeId(), "Reconnection requested for node. Setting status to connecting."); + + // determine if this node should be assigned the primary role + if (primaryNodeId == null || primaryNodeId.logicallyEquals(node.getNodeId())) { + setPrimaryNodeId(node.getNodeId()); + addEvent(node.getNodeId(), "Setting primary role in reconnection request."); + primaryRole = true; + } else { + primaryRole = false; + } + + if (dataFlowManagementService.getRetrievalDelaySeconds() <= 0) { + tryAgainSeconds = DEFAULT_CONNECTION_REQUEST_TRY_AGAIN_SECONDS; + } else { + tryAgainSeconds = dataFlowManagementService.getRetrievalDelaySeconds(); + } + } catch (final UnknownNodeException | IllegalNodeReconnectionException | NodeReconnectionException une) { + throw une; + } catch (final Exception ex) { + logger.warn("Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + ex, ex); + + node.setStatus(Status.DISCONNECTED); + final String eventMsg = "Problem encountered issuing reconnection request. Node will remain disconnected: " + ex; + addEvent(node.getNodeId(), eventMsg); + addBulletin(node, Severity.WARNING, eventMsg); + + // Exception thrown will include node ID but event/bulletin do not because the node/id is passed along with the message + throw new NodeReconnectionException("Problem encountered issuing reconnection request to " + node.getNodeId() + ". Node will remain disconnected: " + ex, ex); + } finally { + writeLock.unlock("requestReconnection"); + } + + // Asynchronously start attempting reconnection. This is not completely thread-safe, as + // we do this by releasing the write lock and then obtaining a read lock for each attempt, + // so we suffer from the ABA problem. However, we are willing to accept the consequences of + // this situation in order to avoid holding a lock for the entire duration. "The consequences" + // are that a second thread could potentially be doing the same thing, issuing a reconnection request. + // However, this is very unlikely to happen, based on the conditions under which we issue a reconnection + // request. And if we do, the node will simply reconnect multiple times, which is not a big deal. + requestReconnectionAsynchronously(node, primaryRole, 10, tryAgainSeconds); + } + + private void requestReconnectionAsynchronously(final Node node, final boolean primaryRole, final int reconnectionAttempts, final int retrySeconds) { + final Thread reconnectionThread = new Thread(new Runnable() { + @Override + public void run() { + for (int i = 0; i < reconnectionAttempts; i++) { + final ReconnectionRequestMessage request = new ReconnectionRequestMessage(); + + try { + readLock.lock(); + try { + if (Status.CONNECTING != node.getStatus()) { + // the node status has changed. It's no longer appropriate to attempt reconnection. + return; + } + + // create the request + request.setNodeId(node.getNodeId()); + request.setDataFlow(cachedDataFlow); + request.setPrimary(primaryRole); + request.setManagerRemoteSiteCommsSecure(remoteCommsSecure); + request.setManagerRemoteSiteListeningPort(remoteInputPort); + request.setInstanceId(instanceId); + } finally { + readLock.unlock("Reconnect " + node.getNodeId()); + } + + // Issue a reconnection request to the node. + senderListener.requestReconnection(request); + + node.setConnectionRequestedTimestamp(System.currentTimeMillis()); + + // successfully told node to reconnect -- we're done! + return; + } catch (final Exception e) { + logger.warn("Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + e); + if (logger.isDebugEnabled()) { + logger.warn("", e); + } + + addBulletin(node, Severity.WARNING, "Problem encountered issuing reconnection request to node " + node.getNodeId() + " due to: " + e); + } + + try { + Thread.sleep(1000L * retrySeconds); + } catch (final InterruptedException ie) { + break; + } + } + + // We failed to reconnect 10 times. We must now mark node as disconnected. + writeLock.lock(); + try { + if (Status.CONNECTING == node.getStatus()) { + requestDisconnectionQuietly(node.getNodeId(), "Failed to issue Reconnection Request " + reconnectionAttempts + " times"); + } + } finally { + writeLock.unlock("Mark node as Disconnected as a result of reconnection failure"); + } + } + }, "Reconnect " + node.getNodeId()); + + reconnectionThread.start(); + } + + private List loadReportingTasks(final File taskConfigXml) { + final List tasks = new ArrayList<>(); + if (taskConfigXml == null) { + logger.info("No controller tasks to start"); + return tasks; + } + + try { + final URL schemaUrl = getClass().getResource("/ReportingTaskConfiguration.xsd"); + final Document document = parse(taskConfigXml, schemaUrl); + + final NodeList tasksNodes = document.getElementsByTagName("tasks"); + final Element tasksElement = (Element) tasksNodes.item(0); + + //optional properties for all ReportingTasks + for (final Element taskElement : DomUtils.getChildElementsByTagName(tasksElement, "task")) { + //add global properties common to all tasks + Map properties = new HashMap<>(); + + //get properties for the specific reporting task - id, name, class, + //and schedulingPeriod must be set + final String taskId = DomUtils.getChild(taskElement, "id").getTextContent().trim(); + final String taskName = DomUtils.getChild(taskElement, "name").getTextContent().trim(); + + final List schedulingStrategyNodeList = DomUtils.getChildElementsByTagName(taskElement, "schedulingStrategy"); + String schedulingStrategyValue = SchedulingStrategy.TIMER_DRIVEN.name(); + if (schedulingStrategyNodeList.size() == 1) { + final String specifiedValue = schedulingStrategyNodeList.get(0).getTextContent(); + + try { + schedulingStrategyValue = SchedulingStrategy.valueOf(specifiedValue).name(); + } catch (final Exception e) { + throw new RuntimeException("Cannot start Reporting Task with id " + taskId + " because its Scheduling Strategy does not have a valid value", e); + } + } + + final SchedulingStrategy schedulingStrategy = SchedulingStrategy.valueOf(schedulingStrategyValue); + final String taskSchedulingPeriod = DomUtils.getChild(taskElement, "schedulingPeriod").getTextContent().trim(); + final String taskClass = DomUtils.getChild(taskElement, "class").getTextContent().trim(); + + //optional task-specific properties + for (final Element optionalProperty : DomUtils.getChildElementsByTagName(taskElement, "property")) { + final String name = optionalProperty.getAttribute("name"); + final String value = optionalProperty.getTextContent().trim(); + properties.put(name, value); + } + + //set the class to be used for the configured reporting task + final ReportingTaskNode reportingTaskNode; + try { + reportingTaskNode = createReportingTask(taskClass, taskId); + } catch (final ReportingTaskInstantiationException e) { + logger.error("Unable to load reporting task {} due to {}", new Object[]{taskId, e}); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + continue; + } + + final ReportingTask reportingTask = reportingTaskNode.getReportingTask(); + + final ReportingInitializationContext config = new StandardReportingInitializationContext(taskId, taskName, schedulingStrategy, taskSchedulingPeriod, this); + reportingTask.initialize(config); + + final Map resolvedProps; + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + resolvedProps = new HashMap<>(); + for (final Map.Entry entry : properties.entrySet()) { + final PropertyDescriptor descriptor = reportingTask.getPropertyDescriptor(entry.getKey()); + resolvedProps.put(descriptor, entry.getValue()); + } + } + + for (final Map.Entry entry : resolvedProps.entrySet()) { + reportingTaskNode.setProperty(entry.getKey().getName(), entry.getValue()); + } + + processScheduler.schedule(reportingTaskNode); + tasks.add(reportingTaskNode); + } + } catch (final SAXException | ParserConfigurationException | IOException | DOMException | NumberFormatException | InitializationException t) { + logger.error("Unable to load reporting tasks from {} due to {}", new Object[]{taskConfigXml, t}); + if (logger.isDebugEnabled()) { + logger.error("", t); + } + } + + return tasks; + } + + private ReportingTaskNode createReportingTask(final String type, final String id) throws ReportingTaskInstantiationException { + if (type == null) { + throw new NullPointerException(); + } + ReportingTask task = null; + final ClassLoader ctxClassLoader = Thread.currentThread().getContextClassLoader(); + try { + final ClassLoader detectedClassLoader = ExtensionManager.getClassLoader(type); + final Class rawClass; + if (detectedClassLoader == null) { + rawClass = Class.forName(type); + } else { + rawClass = Class.forName(type, false, detectedClassLoader); + } + + Thread.currentThread().setContextClassLoader(detectedClassLoader); + final Class reportingTaskClass = rawClass.asSubclass(ReportingTask.class); + final Object reportingTaskObj = reportingTaskClass.newInstance(); + task = reportingTaskClass.cast(reportingTaskObj); + } catch (final ClassNotFoundException | SecurityException | InstantiationException | IllegalAccessException | IllegalArgumentException t) { + throw new ReportingTaskInstantiationException(type, t); + } finally { + if (ctxClassLoader != null) { + Thread.currentThread().setContextClassLoader(ctxClassLoader); + } + } + + final ValidationContextFactory validationContextFactory = new StandardValidationContextFactory(this); + final ReportingTaskNode taskNode = new ClusteredReportingTaskNode(task, id, processScheduler, + new ClusteredEventAccess(this), bulletinRepository, controllerServiceProvider, validationContextFactory); + return taskNode; + } + + private Document parse(final File xmlFile, final URL schemaUrl) throws SAXException, ParserConfigurationException, IOException { + final SchemaFactory schemaFactory = SchemaFactory.newInstance(XMLConstants.W3C_XML_SCHEMA_NS_URI); + final Schema schema = schemaFactory.newSchema(schemaUrl); + final DocumentBuilderFactory docFactory = DocumentBuilderFactory.newInstance(); + docFactory.setSchema(schema); + final DocumentBuilder builder = docFactory.newDocumentBuilder(); + + builder.setErrorHandler(new org.xml.sax.ErrorHandler() { + @Override + public void fatalError(final SAXParseException err) throws SAXException { + logger.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); + if (logger.isDebugEnabled()) { + logger.error("Error Stack Dump", err); + } + throw err; + } + + @Override + public void error(final SAXParseException err) throws SAXParseException { + logger.error("Config file line " + err.getLineNumber() + ", col " + err.getColumnNumber() + ", uri " + err.getSystemId() + " :message: " + err.getMessage()); + if (logger.isDebugEnabled()) { + logger.error("Error Stack Dump", err); + } + throw err; + } + + @Override + public void warning(final SAXParseException err) throws SAXParseException { + logger.warn(" Config file line " + err.getLineNumber() + ", uri " + err.getSystemId() + " : message : " + err.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn("Warning stack dump", err); + } + throw err; + } + }); + + // build the docuemnt + final Document document = builder.parse(xmlFile); + + // ensure schema compliance + final Validator validator = schema.newValidator(); + validator.validate(new DOMSource(document)); + + return document; + } + + private void addBulletin(final Node node, final Severity severity, final String msg) { + addBulletin(node.getNodeId(), severity, msg); + } + + private void addBulletin(final NodeIdentifier nodeId, final Severity severity, final String msg) { + bulletinRepository.addBulletin(BulletinFactory.createBulletin(BULLETIN_CATEGORY, severity.toString(), + nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg)); + } + + /** + * Services a disconnection request. + * + * @param nodeId a node identifier + * @param userDn the DN of the user requesting the disconnection + * + * @throws UnknownNodeException if the node does not exist + * @throws IllegalNodeDisconnectionException if the node cannot be + * disconnected due to the cluster's state (e.g., node is last connected + * node or node is primary) + * @throws NodeDisconnectionException if the disconnection message fails to + * be sent. + */ + @Override + public void requestDisconnection(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeDisconnectionException, NodeDisconnectionException { + writeLock.lock(); + try { + // check that the node is known + final Node node = getNode(nodeId); + if (node == null) { + throw new UnknownNodeException("Node does not exist."); + } + requestDisconnection(node.getNodeId(), /* ignore last node */ false, "User " + userDn + " Disconnected Node"); + } finally { + writeLock.unlock("requestDisconnection(String)"); + } + } + + /** + * Requests a disconnection to the node with the given node ID, but any + * exception thrown is suppressed. + * + * @param nodeId the node ID + */ + private void requestDisconnectionQuietly(final NodeIdentifier nodeId, final String explanation) { + try { + requestDisconnection(nodeId, /* ignore node check */ true, explanation); + } catch (final IllegalNodeDisconnectionException | NodeDisconnectionException ex) { /* suppress exception */ } + } + + /** + * Issues a disconnection message to the node identified by the given node + * ID. If the node is not known, then a UnknownNodeException is thrown. If + * the node cannot be disconnected due to the cluster's state and + * ignoreLastNodeCheck is false, then a IllegalNodeDisconnectionException is + * thrown. Otherwise, a disconnection message is issued to the node. + * + * Whether the disconnection message is successfully sent to the node, the + * node is marked as disconnected and if the node is the primary node, then + * the primary role is revoked. + * + * @param nodeId the ID of the node + * @param ignoreNodeChecks if false, checks will be made to ensure the + * cluster supports the node's disconnection (e.g., the node is not the last + * connected node in the cluster; the node is not the primary); otherwise, + * the request is made regardless of the cluster state + * @param explanation + * + * @throws IllegalNodeDisconnectionException if the node cannot be + * disconnected due to the cluster's state (e.g., node is last connected + * node or node is primary). Not thrown if ignoreNodeChecks is true. + * @throws NodeDisconnectionException if the disconnection message fails to + * be sent. + */ + private void requestDisconnection(final NodeIdentifier nodeId, final boolean ignoreNodeChecks, final String explanation) + throws IllegalNodeDisconnectionException, NodeDisconnectionException { + + writeLock.lock(); + try { + + // check that the node is known + final Node node = getRawNode(nodeId.getId()); + if (node == null) { + if (ignoreNodeChecks) { + // issue the disconnection + final DisconnectMessage request = new DisconnectMessage(); + request.setNodeId(nodeId); + request.setExplanation(explanation); + + addEvent(nodeId, "Disconnection requested due to " + explanation); + senderListener.disconnect(request); + addEvent(nodeId, "Node disconnected due to " + explanation); + addBulletin(nodeId, Severity.INFO, "Node disconnected due to " + explanation); + return; + } else { + throw new UnknownNodeException("Node does not exist"); + } + } + + // if necessary, check that the node may be disconnected + if (!ignoreNodeChecks) { + final Set connectedNodes = getNodeIds(Status.CONNECTED); + // cannot disconnect the last connected node in the cluster + if (connectedNodes.size() == 1 && connectedNodes.iterator().next().equals(nodeId)) { + throw new IllegalNodeDisconnectionException("Node may not be disconnected because it is the only connected node in the cluster."); + } else if (isPrimaryNode(nodeId)) { + // cannot disconnect the primary node in the cluster + throw new IllegalNodeDisconnectionException("Node may not be disconnected because it is the primary node in the cluster."); + } + } + + // update status + node.setStatus(Status.DISCONNECTED); + notifyDataFlowManagementServiceOfNodeStatusChange(); + + // issue the disconnection + final DisconnectMessage request = new DisconnectMessage(); + request.setNodeId(nodeId); + request.setExplanation(explanation); + + addEvent(nodeId, "Disconnection requested due to " + explanation); + senderListener.disconnect(request); + addEvent(nodeId, "Node disconnected due to " + explanation); + addBulletin(node, Severity.INFO, "Node disconnected due to " + explanation); + } finally { + writeLock.unlock("requestDisconnection(NodeIdentifier, boolean)"); + } + } + + /** + * Messages the node to have the primary role. If the messaging fails, then + * the node is marked as disconnected. + * + * @param nodeId the node ID to assign primary role + * + * @return true if primary role assigned; false otherwise + */ + private boolean assignPrimaryRole(final NodeIdentifier nodeId) { + writeLock.lock(); + try { + // create primary role message + final PrimaryRoleAssignmentMessage msg = new PrimaryRoleAssignmentMessage(); + msg.setNodeId(nodeId); + msg.setPrimary(true); + logger.info("Attempting to assign primary role to node: " + nodeId); + + // message + senderListener.assignPrimaryRole(msg); + + logger.info("Assigned primary role to node: " + nodeId); + addBulletin(nodeId, Severity.INFO, "Node assigned primary role"); + + // true indicates primary role assigned + return true; + + } catch (final ProtocolException ex) { + + logger.warn("Failed attempt to assign primary role to node " + nodeId + " due to " + ex); + addBulletin(nodeId, Severity.ERROR, "Failed to assign primary role to node due to: " + ex); + + // mark node as disconnected and log/record the event + final Node node = getRawNode(nodeId.getId()); + node.setStatus(Status.DISCONNECTED); + addEvent(node.getNodeId(), "Disconnected because of failed attempt to assign primary role."); + + addBulletin(nodeId, Severity.WARNING, "Node disconnected because of failed attempt to assign primary role"); + + // false indicates primary role failed to be assigned + return false; + } finally { + writeLock.unlock("assignPrimaryRole"); + } + } + + /** + * Messages the node with the given node ID to no longer have the primary + * role. If the messaging fails, then the node is marked as disconnected. + * + * @return true if the primary role was revoked from the node; false + * otherwise + */ + private boolean revokePrimaryRole(final NodeIdentifier nodeId) { + writeLock.lock(); + try { + // create primary role message + final PrimaryRoleAssignmentMessage msg = new PrimaryRoleAssignmentMessage(); + msg.setNodeId(nodeId); + msg.setPrimary(false); + logger.info("Attempting to revoke primary role from node: " + nodeId); + + // send message + senderListener.assignPrimaryRole(msg); + + logger.info("Revoked primary role from node: " + nodeId); + addBulletin(nodeId, Severity.INFO, "Primary Role revoked from node"); + + // true indicates primary role was revoked + return true; + } catch (final ProtocolException ex) { + + logger.warn("Failed attempt to revoke primary role from node " + nodeId + " due to " + ex); + + // mark node as disconnected and log/record the event + final Node node = getRawNode(nodeId.getId()); + node.setStatus(Status.DISCONNECTED); + addEvent(node.getNodeId(), "Disconnected because of failed attempt to revoke primary role."); + addBulletin(node, Severity.ERROR, "Node disconnected because of failed attempt to revoke primary role"); + + // false indicates primary role failed to be revoked + return false; + } finally { + writeLock.unlock("revokePrimaryRole"); + } + } + + private NodeIdentifier addRequestorDn(final NodeIdentifier nodeId, final String dn) { + return new NodeIdentifier(nodeId.getId(), nodeId.getApiAddress(), + nodeId.getApiPort(), nodeId.getSocketAddress(), nodeId.getSocketPort(), dn); + } + + private ConnectionResponseMessage handleConnectionRequest(final ConnectionRequestMessage requestMessage) { + final NodeIdentifier proposedIdentifier = requestMessage.getConnectionRequest().getProposedNodeIdentifier(); + final ConnectionRequest requestWithDn = new ConnectionRequest(addRequestorDn(proposedIdentifier, requestMessage.getRequestorDN())); + + final ConnectionResponse response = requestConnection(requestWithDn); + final ConnectionResponseMessage responseMessage = new ConnectionResponseMessage(); + responseMessage.setConnectionResponse(response); + return responseMessage; + } + + private void handleControllerStartupFailure(final ControllerStartupFailureMessage msg) { + writeLock.lock(); + try { + final Node node = getRawNode(msg.getNodeId().getId()); + if (node != null) { + node.setStatus(Status.DISCONNECTED); + addEvent(msg.getNodeId(), "Node could not join cluster because it failed to start up properly. Setting node to Disconnected. Node reported the following error: " + msg.getExceptionMessage()); + addBulletin(node, Severity.ERROR, "Node could not join cluster because it failed to start up properly. Setting node to Disconnected. Node reported the following error: " + msg.getExceptionMessage()); + } + } finally { + writeLock.unlock("handleControllerStartupFailure"); + } + } + + private void handleReconnectionFailure(final ReconnectionFailureMessage msg) { + writeLock.lock(); + try { + final Node node = getRawNode(msg.getNodeId().getId()); + if (node != null) { + node.setStatus(Status.DISCONNECTED); + final String errorMsg = "Node could not rejoin cluster. Setting node to Disconnected. Node reported the following error: " + msg.getExceptionMessage(); + addEvent(msg.getNodeId(), errorMsg); + addBulletin(node, Severity.ERROR, errorMsg); + } + } finally { + writeLock.unlock("handleControllerStartupFailure"); + } + } + + /** + * Adds an instance of a specified controller service. + * + * @param type + * @param id + * @param properties + * @return + */ + @Override + public ControllerServiceNode createControllerService(String type, String id, Map properties) { + return controllerServiceProvider.createControllerService(type, id, properties); + } + + @Override + public ControllerService getControllerService(String serviceIdentifier) { + return controllerServiceProvider.getControllerService(serviceIdentifier); + } + + @Override + public ControllerServiceNode getControllerServiceNode(final String id) { + return controllerServiceProvider.getControllerServiceNode(id); + } + + @Override + public boolean isControllerServiceEnabled(final ControllerService service) { + return controllerServiceProvider.isControllerServiceEnabled(service); + } + + @Override + public boolean isControllerServiceEnabled(final String serviceIdentifier) { + return controllerServiceProvider.isControllerServiceEnabled(serviceIdentifier); + } + + /** + * Handle a bulletins message. + * + * @param bulletins + */ + public void handleBulletins(final NodeBulletins bulletins) { + final NodeIdentifier nodeIdentifier = bulletins.getNodeIdentifier(); + final String nodeAddress = nodeIdentifier.getApiAddress() + ":" + nodeIdentifier.getApiPort(); + + // unmarshal the message + BulletinsPayload payload = BulletinsPayload.unmarshal(bulletins.getPayload()); + for (final Bulletin bulletin : payload.getBulletins()) { + bulletin.setNodeAddress(nodeAddress); + bulletinRepository.addBulletin(bulletin); + } + } + + /** + * Handles a node's heartbeat. If this heartbeat is a node's first heartbeat + * since its connection request, then the manager will mark the node as + * connected. If the node was previously disconnected due to a lack of + * heartbeat, then a reconnection request is issued. If the node was + * disconnected for other reasons, then a disconnection request is issued. + * If this instance is configured with a firewall and the heartbeat is + * blocked, then a disconnection request is issued. + * + * @param heartbeat + */ + @Override + public void handleHeartbeat(final Heartbeat heartbeat) { + // sanity check heartbeat + if (heartbeat == null) { + throw new IllegalArgumentException("Heartbeat may not be null."); + } else if (heartbeat.getNodeIdentifier() == null) { + throw new IllegalArgumentException("Heartbeat does not contain a node ID."); + } + + /* + * Processing a heartbeat requires a write lock, which may take a while + * to obtain. Only the last heartbeat is necessary to process per node. + * Futhermore, since many could pile up, heartbeats are processed in + * bulk. + * + * The below queue stores the pending heartbeats. + */ + pendingHeartbeats.add(heartbeat); + } + + private void processPendingHeartbeats() { + Node node; + + writeLock.lock(); + try { + /* + * Get the most recent heartbeats for the nodes in the cluster. This + * is achieved by "draining" the pending heartbeats queue, populating + * a map that associates a node identifier with its latest heartbeat, and + * finally, getting the values of the map. + */ + final Map mostRecentHeartbeatsMap = new HashMap<>(); + Heartbeat aHeartbeat; + while ((aHeartbeat = pendingHeartbeats.poll()) != null) { + mostRecentHeartbeatsMap.put(aHeartbeat.getNodeIdentifier(), aHeartbeat); + } + final Collection mostRecentHeartbeats = new ArrayList<>(mostRecentHeartbeatsMap.values()); + + // return fast if no work to do + if (mostRecentHeartbeats.isEmpty()) { + return; + } + + logNodes("Before Heartbeat Processing", heartbeatLogger); + + final int numPendingHeartbeats = mostRecentHeartbeats.size(); + if (heartbeatLogger.isDebugEnabled()) { + heartbeatLogger.debug(String.format("Handling %s heartbeat%s", numPendingHeartbeats, (numPendingHeartbeats > 1) ? "s" : "")); + } + + for (final Heartbeat mostRecentHeartbeat : mostRecentHeartbeats) { + try { + // resolve the proposed node identifier to valid node identifier + final NodeIdentifier resolvedNodeIdentifier = resolveProposedNodeIdentifier(mostRecentHeartbeat.getNodeIdentifier()); + + // get a raw reference to the node (if it doesn't exist, node will be null) + node = getRawNode(resolvedNodeIdentifier.getId()); + + // if the node thinks it has the primary role, but the manager has assigned the role to a different node, then revoke the role + if (mostRecentHeartbeat.isPrimary() && !isPrimaryNode(resolvedNodeIdentifier)) { + addEvent(resolvedNodeIdentifier, "Heartbeat indicates node is running as primary node. Revoking primary role because primary role is assigned to a different node."); + revokePrimaryRole(resolvedNodeIdentifier); + } + + final boolean heartbeatIndicatesNotYetConnected = !mostRecentHeartbeat.isConnected(); + + if (isBlockedByFirewall(resolvedNodeIdentifier.getSocketAddress())) { + if (node == null) { + logger.info("Firewall blocked heartbeat received from unknown node " + resolvedNodeIdentifier + ". Issuing disconnection request."); + } else { + // record event + addEvent(resolvedNodeIdentifier, "Firewall blocked received heartbeat. Issuing disconnection request."); + } + + // request node to disconnect + requestDisconnectionQuietly(resolvedNodeIdentifier, "Blocked By Firewall"); + + } else if (node == null) { // unknown node, so issue reconnect request + // create new node and add to node set + final Node newNode = new Node(resolvedNodeIdentifier, Status.DISCONNECTED); + nodes.add(newNode); + + // record event + addEvent(newNode.getNodeId(), "Received heartbeat from unknown node. Issuing reconnection request."); + + // record heartbeat + newNode.setHeartbeat(mostRecentHeartbeat); + requestReconnection(resolvedNodeIdentifier.getId(), "NCM Heartbeat Processing"); + } else if (heartbeatIndicatesNotYetConnected) { + if (Status.CONNECTED == node.getStatus()) { + // record event + addEvent(node.getNodeId(), "Received heartbeat from node that thinks it is not yet part of the cluster, though the Manager thought it was. Marking as Disconnected and issuing reconnection request."); + + // record heartbeat + node.setHeartbeat(null); + node.setStatus(Status.DISCONNECTED); + + requestReconnection(resolvedNodeIdentifier.getId(), "NCM Heartbeat Processing"); + } + } else if (Status.DISCONNECTED == node.getStatus()) { + // ignore heartbeats from nodes disconnected by means other than lack of heartbeat, unless it is + // the only node. We allow it if it is the only node because if we have a one-node cluster, then + // we cannot manually reconnect it. + if (node.isHeartbeatDisconnection() || nodes.size() == 1) { + // record event + if (node.isHeartbeatDisconnection()) { + addEvent(resolvedNodeIdentifier, "Received heartbeat from node previously disconnected due to lack of heartbeat. Issuing reconnection request."); + } else { + addEvent(resolvedNodeIdentifier, "Received heartbeat from node previously disconnected, but it is the only known node, so issuing reconnection request."); + } + + // record heartbeat + node.setHeartbeat(mostRecentHeartbeat); + + // request reconnection + requestReconnection(resolvedNodeIdentifier.getId(), "NCM Heartbeat Processing"); + } else { + // disconnected nodes should not heartbeat, so we need to issue a disconnection request + heartbeatLogger.info("Ignoring received heartbeat from disconnected node " + resolvedNodeIdentifier + ". Issuing disconnection request."); + + // request node to disconnect + requestDisconnectionQuietly(resolvedNodeIdentifier, "Received Heartbeat from Node, but Manager has already marked Node as Disconnected"); + } + + } else if (Status.DISCONNECTING == node.getStatus()) { + /* ignore spurious heartbeat */ + } else { // node is either either connected or connecting + // first heartbeat causes status change from connecting to connected + if (Status.CONNECTING == node.getStatus()) { + if (mostRecentHeartbeat.getCreatedTimestamp() < node.getConnectionRequestedTimestamp()) { + heartbeatLogger.info("Received heartbeat for node " + resolvedNodeIdentifier + " but ignoring because it was generated before the node was last asked to reconnect."); + continue; + } + + // set status to connected + node.setStatus(Status.CONNECTED); + + // record event + addEvent(resolvedNodeIdentifier, "Received first heartbeat from connecting node. Setting node to connected."); + + // notify service of updated node set + notifyDataFlowManagementServiceOfNodeStatusChange(); + + addBulletin(node, Severity.INFO, "Node Connected"); + } else { + heartbeatLogger.info("Received heartbeat for node " + resolvedNodeIdentifier + "."); + } + + // record heartbeat + node.setHeartbeat(mostRecentHeartbeat); + + ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); + if (statusRepository == null) { + statusRepository = createComponentStatusRepository(); + componentMetricsRepositoryMap.put(node.getNodeId(), statusRepository); + } + + // If it's been a while since we've captured, capture this metric. + final Date lastCaptureDate = statusRepository.getLastCaptureDate(); + final long millisSinceLastCapture = (lastCaptureDate == null) ? Long.MAX_VALUE : (System.currentTimeMillis() - lastCaptureDate.getTime()); + + if (millisSinceLastCapture > componentStatusSnapshotMillis) { + statusRepository.capture(node.getHeartbeatPayload().getProcessGroupStatus()); + } + } + } catch (final Exception e) { + logger.error("Failed to process heartbeat from {}:{} due to {}", mostRecentHeartbeat.getNodeIdentifier().getApiAddress(), mostRecentHeartbeat.getNodeIdentifier().getApiPort(), e.toString()); + if (logger.isDebugEnabled()) { + logger.error("", e); + } + } + } + + logNodes("After Heartbeat Processing", heartbeatLogger); + } finally { + writeLock.unlock("processPendingHeartbeats"); + } + } + + private ComponentStatusRepository createComponentStatusRepository() { + final String implementationClassName = properties.getProperty(NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION, DEFAULT_COMPONENT_STATUS_REPO_IMPLEMENTATION); + if (implementationClassName == null) { + throw new RuntimeException("Cannot create Component Status Repository because the NiFi Properties is missing the following property: " + + NiFiProperties.COMPONENT_STATUS_REPOSITORY_IMPLEMENTATION); + } + + try { + return NarThreadContextClassLoader.createInstance(implementationClassName, ComponentStatusRepository.class); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } + + @Override + public Set getNodes(final Status... statuses) { + final Set desiredStatusSet = new HashSet<>(); + for (final Status status : statuses) { + desiredStatusSet.add(status); + } + + readLock.lock(); + try { + final Set clonedNodes = new HashSet<>(); + for (final Node node : nodes) { + if (desiredStatusSet.isEmpty() || desiredStatusSet.contains(node.getStatus())) { + clonedNodes.add(node.clone()); + } + } + return Collections.unmodifiableSet(clonedNodes); + } finally { + readLock.unlock("getNodes(Status...)"); + } + } + + @Override + public Node getNode(final String nodeId) { + readLock.lock(); + try { + for (final Node node : nodes) { + if (node.getNodeId().getId().equals(nodeId)) { + return node.clone(); + } + } + return null; + } finally { + readLock.unlock("getNode(String)"); + } + } + + @Override + public Node getPrimaryNode() { + readLock.lock(); + try { + if (primaryNodeId == null) { + return null; + } else { + return getNode(primaryNodeId.getId()); + } + } finally { + readLock.unlock("getPrimaryNode"); + } + } + + @Override + public void deleteNode(final String nodeId, final String userDn) throws UnknownNodeException, IllegalNodeDeletionException { + writeLock.lock(); + try { + final Node node = getNode(nodeId); + if (node == null) { + throw new UnknownNodeException("Node does not exist."); + } else if (Status.DISCONNECTED == node.getStatus()) { + nodes.remove(node); + + if (eventManager != null) { + eventManager.clearEventHistory(node.getNodeId().getId()); + } + + logger.info("Removing node {} from cluster because this action was requested by {}", node, userDn); + } else { + throw new IllegalNodeDeletionException("Node may not be deleted because it is not disconnected."); + } + } finally { + writeLock.unlock("deleteNode"); + } + } + + @Override + public Set getNodeIds(final Status... statuses) { + readLock.lock(); + try { + final Set nodeIds = new HashSet<>(); + for (final Node node : nodes) { + if (statuses == null || statuses.length == 0) { + nodeIds.add(node.getNodeId()); + } else { + for (final Node.Status status : statuses) { + if (node.getStatus() == status) { + nodeIds.add(node.getNodeId()); + break; + } + } + } + } + return nodeIds; + } finally { + readLock.unlock("getNodeIds(Status...)"); + } + } + + @Override + public void setPrimaryNode(final String nodeId, final String userDn) throws UnknownNodeException, IneligiblePrimaryNodeException, PrimaryRoleAssignmentException { + writeLock.lock(); + try { + + final Node node = getNode(nodeId); + if (node == null) { + throw new UnknownNodeException("Node does not exist."); + } else if (Status.CONNECTED != node.getStatus()) { + throw new IneligiblePrimaryNodeException("Node must be connected before it can be assigned as the primary node."); + } + + // revoke primary role + final Node primaryNode; + if ((primaryNode = getPrimaryNode()) != null) { + if (primaryNode.getStatus() == Status.DISCONNECTED) { + throw new PrimaryRoleAssignmentException("A disconnected, primary node exists. Delete the node before assigning the primary role to a different node."); + } else if (revokePrimaryRole(primaryNode.getNodeId())) { + addEvent(primaryNode.getNodeId(), "Role revoked from this node as part of primary role reassignment."); + } else { + throw new PrimaryRoleAssignmentException( + "Failed to revoke primary role from node. Primary node is now disconnected. Delete the node before assigning the primary role to a different node."); + } + } + + // change the primary node ID to the given node + setPrimaryNodeId(node.getNodeId()); + + // assign primary role + if (assignPrimaryRole(node.getNodeId())) { + addEvent(node.getNodeId(), "Role assigned to this node as part of primary role reassignment. Action performed by " + userDn); + addBulletin(node, Severity.INFO, "Primary Role assigned to node by " + userDn); + } else { + throw new PrimaryRoleAssignmentException( + "Cluster manager assigned primary role to node, but the node failed to accept the assignment. Cluster manager disconnected node."); + } + } finally { + writeLock.unlock("setPrimaryNode"); + } + } + + private int getClusterProtocolHeartbeatSeconds() { + return (int) FormatUtils.getTimeDuration(properties.getClusterProtocolHeartbeatInterval(), TimeUnit.SECONDS); + } + + @Override + public int getHeartbeatMonitoringIntervalSeconds() { + return 4 * getClusterProtocolHeartbeatSeconds(); + } + + @Override + public int getMaxHeartbeatGapSeconds() { + return 8 * getClusterProtocolHeartbeatSeconds(); + } + + @Override + public List getNodeEvents(final String nodeId) { + readLock.lock(); + try { + List events = null; + final EventManager eventMgr = eventManager; + if (eventMgr != null) { + events = eventMgr.getEvents(nodeId); + } + + if (events == null) { + return Collections.emptyList(); + } else { + return Collections.unmodifiableList(events); + } + } finally { + readLock.unlock("getNodeEvents"); + } + } + + @Override + public NodeResponse applyRequest(final String method, final URI uri, final Map> parameters, final Map headers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { + return applyRequest(method, uri, parameters, headers, getNodeIds(Status.CONNECTED)); + } + + @Override + public NodeResponse applyRequest(final String method, final URI uri, final Map> parameters, final Map headers, final Set nodeIdentifiers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { + + final boolean mutableRequest = canChangeNodeState(method, uri); + final ClusterManagerLock lock = mutableRequest ? writeLock : readLock; + + lock.lock(); + try { + // check that the request can be applied + if (mutableRequest) { + if (isInSafeMode()) { + throw new SafeModeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while in safe mode"); + } else if (!getNodeIds(Status.DISCONNECTED, Status.DISCONNECTING).isEmpty()) { + throw new DisconnectedNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is disconnected from the cluster"); + } else if (!getNodeIds(Status.CONNECTING).isEmpty()) { + // if any node is connecting and a request can change the flow, then we throw an exception + throw new ConnectingNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is trying to connect to the cluster"); + } + } + + final NodeResponse clientResponse = federateRequest(method, uri, parameters, null, headers, nodeIdentifiers); + if (clientResponse == null) { + if (mutableRequest) { + throw new NoConnectedNodesException(String.format("All nodes were disconnected as a result of applying request %s %s", method, uri)); + } else { + throw new NoResponseFromNodesException("No nodes were able to process this request."); + } + } else { + return clientResponse; + } + } finally { + lock.unlock("applyRequest(String, URI, Map>, Map, Set"); + } + } + + @Override + public NodeResponse applyRequest(final String method, final URI uri, final Object entity, final Map headers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { + return applyRequest(method, uri, entity, headers, getNodeIds(Status.CONNECTED)); + } + + @Override + public NodeResponse applyRequest(final String method, final URI uri, final Object entity, final Map headers, final Set nodeIdentifiers) + throws NoConnectedNodesException, NoResponseFromNodesException, UriConstructionException, ConnectingNodeMutableRequestException, DisconnectedNodeMutableRequestException, SafeModeMutableRequestException { + + final boolean mutableRequest = canChangeNodeState(method, uri); + final ClusterManagerLock lock = mutableRequest ? writeLock : readLock; + + lock.lock(); + try { + // check that the request can be applied + if (mutableRequest) { + if (isInSafeMode()) { + throw new SafeModeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while in safe mode"); + } else if (!getNodeIds(Status.DISCONNECTED, Status.DISCONNECTING).isEmpty()) { + throw new DisconnectedNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is disconnected from the cluster"); + } else if (!getNodeIds(Status.CONNECTING).isEmpty()) { + // if any node is connecting and a request can change the flow, then we throw an exception + throw new ConnectingNodeMutableRequestException("Received a mutable request [" + method + " -- " + uri + "] while a node is trying to connect to the cluster"); + } + } + + final NodeResponse clientResponse = federateRequest(method, uri, null, entity, headers, nodeIdentifiers); + if (clientResponse == null) { + if (mutableRequest) { + throw new NoConnectedNodesException(String.format("All nodes were disconnected as a result of applying request %s %s", method, uri)); + } else { + throw new NoResponseFromNodesException("No nodes were able to process this request."); + } + } else { + return clientResponse; + } + + } finally { + lock.unlock("applyRequest(String, URI, Object, Map, Set"); + } + } + + public void setServicesBroadcaster(final ClusterServicesBroadcaster servicesBroadcaster) { + writeLock.lock(); + try { + this.servicesBroadcaster = servicesBroadcaster; + } finally { + writeLock.unlock("setServicesBroadcaster"); + } + } + + public boolean addBroadcastedService(final DiscoverableService service) { + writeLock.lock(); + try { + final ClusterServicesBroadcaster broadcaster = this.servicesBroadcaster; + if (broadcaster == null) { + throw new IllegalStateException("Service broadcasting is not configured."); + } + return broadcaster.addService(service); + } finally { + writeLock.unlock("addBroadcastedService"); + } + } + + public boolean removeBroadcastedService(final String serviceName) { + writeLock.lock(); + try { + final ClusterServicesBroadcaster broadcaster = this.servicesBroadcaster; + if (broadcaster == null) { + throw new IllegalStateException("Service broadcasting is not configured."); + } + return broadcaster.removeService(serviceName); + } finally { + writeLock.unlock("removeBroadcastedService"); + } + } + + public boolean isBroadcastingConfigured() { + readLock.lock(); + try { + return servicesBroadcaster != null; + } finally { + readLock.unlock("isBroadcastingConfigured"); + } + } + + public boolean isBroadcasting() { + readLock.lock(); + try { + final ClusterServicesBroadcaster broadcaster = this.servicesBroadcaster; + return (broadcaster != null && broadcaster.isRunning()); + } finally { + readLock.unlock("isBroadcasting"); + } + } + + public void addEvent(final NodeIdentifier nodeId, String eventMsg) { + writeLock.lock(); + try { + final Event event = new Event(nodeId.getId(), eventMsg); + final EventManager eventMgr = eventManager; + if (eventMgr != null) { + eventMgr.addEvent(event); + } + logger.info(String.format("Node Event: %s -- '%s'", nodeId, eventMsg)); + } finally { + writeLock.unlock("addEvent"); + } + } + + public void setEventManager(final EventManager eventManager) { + writeLock.lock(); + try { + this.eventManager = eventManager; + } finally { + writeLock.unlock("setEventManager"); + } + } + + public void setClusterFirewall(final ClusterNodeFirewall clusterFirewall) { + writeLock.lock(); + try { + this.clusterFirewall = clusterFirewall; + } finally { + writeLock.unlock("setClusterFirewall"); + } + } + + public boolean isFirewallConfigured() { + readLock.lock(); + try { + return clusterFirewall != null; + } finally { + readLock.unlock("isFirewallConfigured"); + } + } + + public void setAuditService(final AuditService auditService) { + writeLock.lock(); + try { + this.auditService = auditService; + } finally { + writeLock.unlock("setAuditService"); + } + } + + public boolean isAuditingConfigured() { + readLock.lock(); + try { + return auditService != null; + } finally { + readLock.unlock("isAuditingConfigured"); + } + } + + private boolean isPrimaryNode(final NodeIdentifier nodeId) { + readLock.lock(); + try { + return primaryNodeId != null && primaryNodeId.equals(nodeId); + } finally { + readLock.unlock("isPrimaryNode"); + } + } + + private boolean isInSafeMode() { + readLock.lock(); + try { + return primaryNodeId == null || getRawNode(primaryNodeId.getId()) == null; + } finally { + readLock.unlock("isInSafeMode"); + } + } + + private void setPrimaryNodeId(final NodeIdentifier primaryNodeId) throws DaoException { + writeLock.lock(); + try { + dataFlowManagementService.updatePrimaryNode(primaryNodeId); + + // update the cached copy reference to minimize loading file from disk + this.primaryNodeId = primaryNodeId; + } finally { + writeLock.unlock("setPrimaryNodeId"); + } + } + + // requires write lock to already be acquired unless method cannot change node state + private NodeResponse federateRequest(final String method, final URI uri, final Map> parameters, final Object entity, final Map headers, final Set nodeIds) throws UriConstructionException { + // ensure some nodes are connected + if (nodeIds.isEmpty()) { + throw new NoConnectedNodesException("Cannot apply " + method + " request to " + uri + " because there are currently no connected Nodes"); + } + + logger.debug("Applying prototype request " + uri + " to nodes."); + + // the starting state of the flow (current, stale, unknown) + final PersistedFlowState originalPersistedFlowState = dataFlowManagementService.getPersistedFlowState(); + + // check if this request can change the flow + final boolean mutableRequest = canChangeNodeState(method, uri); + + // update headers to contain cluster contextual information to send to the node + final Map updatedHeaders = new HashMap<>(headers); + final ClusterContext clusterCtx = new ClusterContextImpl(); + clusterCtx.setRequestSentByClusterManager(true); // indicate request is sent from cluster manager + clusterCtx.setRevision(revision); + + // serialize cluster context and add to request header + final String serializedClusterCtx = WebUtils.serializeObjectToHex(clusterCtx); + updatedHeaders.put(CLUSTER_CONTEXT_HTTP_HEADER, serializedClusterCtx); + + // if the request is mutable, we need to verify that it is a valid request for all nodes in the cluster. + if (mutableRequest) { + updatedHeaders.put(NCM_EXPECTS_HTTP_HEADER, "150-NodeContinue"); + + final Set nodeResponses; + if (entity == null) { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); + } else { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); + } + + updatedHeaders.remove(NCM_EXPECTS_HTTP_HEADER); + + for (final NodeResponse response : nodeResponses) { + if (response.getStatus() != NODE_CONTINUE_STATUS_CODE) { + final String nodeDescription = response.getNodeId().getApiAddress() + ":" + response.getNodeId().getApiPort(); + final ClientResponse clientResponse = response.getClientResponse(); + if (clientResponse == null) { + throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: Unexpected Response Code " + response.getStatus()); + } + final String nodeExplanation = clientResponse.getEntity(String.class); + throw new IllegalClusterStateException("Node " + nodeDescription + " is unable to fulfill this request due to: " + nodeExplanation, response.getThrowable()); + } + } + + // set flow state to unknown to denote a mutable request replication in progress + logger.debug("Setting Flow State to UNKNOWN due to mutable request to {} {}", method, uri); + notifyDataFlowManagmentServiceOfFlowStateChange(PersistedFlowState.UNKNOWN); + } + + // replicate request + final Set nodeResponses; + try { + if (entity == null) { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, parameters, updatedHeaders); + } else { + nodeResponses = httpRequestReplicator.replicate(nodeIds, method, uri, entity, updatedHeaders); + } + } catch (final UriConstructionException uce) { + // request was not replicated, so mark the flow with its original state + if (mutableRequest) { + notifyDataFlowManagmentServiceOfFlowStateChange(originalPersistedFlowState); + } + + throw uce; + } + + final NodeResponse clientResponse = mergeResponses(uri, method, nodeResponses, mutableRequest); + return clientResponse; + } + + private static boolean isProcessorsEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + private static boolean isProcessorEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PROCESSOR_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && PROCESSORS_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } + + return false; + } + + private static boolean isProcessGroupEndpoint(final URI uri, final String method) { + return ("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && PROCESS_GROUP_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + private static boolean isTemplateEndpoint(final URI uri, final String method) { + return "POST".equalsIgnoreCase(method) && TEMPLATE_INSTANCE_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + private static boolean isFlowSnippetEndpoint(final URI uri, final String method) { + return "POST".equalsIgnoreCase(method) && FLOW_SNIPPET_INSTANCE_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + private static boolean isRemoteProcessGroupsEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && REMOTE_PROCESS_GROUPS_URI_PATTERN.matcher(uri.getPath()).matches(); + } + + private static boolean isRemoteProcessGroupEndpoint(final URI uri, final String method) { + if (("GET".equalsIgnoreCase(method) || "PUT".equalsIgnoreCase(method)) && REMOTE_PROCESS_GROUP_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } else if ("POST".equalsIgnoreCase(method) && REMOTE_PROCESS_GROUPS_URI_PATTERN.matcher(uri.getPath()).matches()) { + return true; + } + + return false; + } + + private static boolean isProvenanceQueryEndpoint(final URI uri, final String method) { + if ("POST".equalsIgnoreCase(method) && PROVENANCE_URI.equals(uri.getPath())) { + return true; + } else if ("GET".equalsIgnoreCase(method) && PROVENANCE_QUERY_URI.matcher(uri.getPath()).matches()) { + return true; + } + return false; + } + + private static boolean isProvenanceEventEndpoint(final URI uri, final String method) { + return "GET".equalsIgnoreCase(method) && PROVENANCE_EVENT_URI.matcher(uri.getPath()).matches(); + } + + static boolean isResponseInterpreted(final URI uri, final String method) { + return isProcessorsEndpoint(uri, method) || isProcessorEndpoint(uri, method) + || isRemoteProcessGroupsEndpoint(uri, method) || isRemoteProcessGroupEndpoint(uri, method) + || isProcessGroupEndpoint(uri, method) + || isTemplateEndpoint(uri, method) || isFlowSnippetEndpoint(uri, method) + || isProvenanceQueryEndpoint(uri, method) || isProvenanceEventEndpoint(uri, method); + } + + private void mergeProcessorValidationErrors(final ProcessorDTO processor, Map processorMap) { + final Map> validationErrorMap = new HashMap<>(); + + for (final Map.Entry nodeEntry : processorMap.entrySet()) { + final NodeIdentifier nodeId = nodeEntry.getKey(); + final ProcessorDTO nodeProcessor = nodeEntry.getValue(); + + // get the processor's validation errors and put them into a map + // where the key is the validation error and the value is the set of all + // nodes that reported that validation error. + final Collection nodeValidationErrors = nodeProcessor.getValidationErrors(); + if (nodeValidationErrors != null) { + for (final String nodeValidationError : nodeValidationErrors) { + Set nodeSet = validationErrorMap.get(nodeValidationError); + if (nodeSet == null) { + nodeSet = new HashSet<>(); + validationErrorMap.put(nodeValidationError, nodeSet); + } + nodeSet.add(nodeId); + } + } + } + + final Set normalizedValidationErrors = new HashSet<>(); + for (final Map.Entry> validationEntry : validationErrorMap.entrySet()) { + final String msg = validationEntry.getKey(); + final Set nodeIds = validationEntry.getValue(); + + if (nodeIds.size() == processorMap.size()) { + normalizedValidationErrors.add(msg); + } else { + for (final NodeIdentifier nodeId : nodeIds) { + normalizedValidationErrors.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + msg); + } + } + } + + processor.setValidationErrors(normalizedValidationErrors); + } + + private void mergeProvenanceQueryResults(final ProvenanceDTO provenanceDto, final Map resultMap, final Set problematicResponses) { + final ProvenanceResultsDTO results = provenanceDto.getResults(); + final ProvenanceRequestDTO request = provenanceDto.getRequest(); + final List allResults = new ArrayList<>(1024); + + final Set errors = new HashSet<>(); + Date oldestEventDate = new Date(); + int percentageComplete = 0; + boolean finished = true; + + long totalRecords = 0; + for (final Map.Entry entry : resultMap.entrySet()) { + final NodeIdentifier nodeIdentifier = entry.getKey(); + final String nodeAddress = nodeIdentifier.getApiAddress() + ":" + nodeIdentifier.getApiPort(); + + final ProvenanceDTO nodeDto = entry.getValue(); + final ProvenanceResultsDTO nodeResultDto = nodeDto.getResults(); + if (nodeResultDto != null && nodeResultDto.getProvenanceEvents() != null) { + // increment the total number of records + totalRecords += nodeResultDto.getTotalCount(); + + // populate the cluster identifier + for (final ProvenanceEventDTO eventDto : nodeResultDto.getProvenanceEvents()) { + eventDto.setClusterNodeId(nodeIdentifier.getId()); + eventDto.setClusterNodeAddress(nodeAddress); + // add node identifier to the event's id so that it is unique across cluster + eventDto.setId(nodeIdentifier.getId() + eventDto.getId()); + allResults.add(eventDto); + } + } + + if (nodeResultDto.getOldestEvent() != null && nodeResultDto.getOldestEvent().before(oldestEventDate)) { + oldestEventDate = nodeResultDto.getOldestEvent(); + } + + if (nodeResultDto.getErrors() != null) { + for (final String error : nodeResultDto.getErrors()) { + errors.add(nodeAddress + " -- " + error); + } + } + + percentageComplete += nodeDto.getPercentCompleted(); + if (!nodeDto.isFinished()) { + finished = false; + } + } + percentageComplete /= resultMap.size(); + + // consider any problematic responses as errors + for (final NodeResponse problematicResponse : problematicResponses) { + final NodeIdentifier problemNode = problematicResponse.getNodeId(); + final String problemNodeAddress = problemNode.getApiAddress() + ":" + problemNode.getApiPort(); + errors.add(String.format("%s -- Request did not complete successfully (Status code: %s)", problemNodeAddress, problematicResponse.getStatus())); + } + + // Since we get back up to the maximum number of results from each node, we need to sort those values and then + // grab only the first X number of them. We do a sort based on time, such that the newest are included. + // If 2 events have the same timestamp, we do a secondary sort based on Cluster Node Identifier. If those are + // equal, we perform a terciary sort based on the the event id + Collections.sort(allResults, new Comparator() { + @Override + public int compare(final ProvenanceEventDTO o1, final ProvenanceEventDTO o2) { + final int eventTimeComparison = o1.getEventTime().compareTo(o2.getEventTime()); + if (eventTimeComparison != 0) { + return -eventTimeComparison; + } + + final String nodeId1 = o1.getClusterNodeId(); + final String nodeId2 = o2.getClusterNodeId(); + final int nodeIdComparison; + if (nodeId1 == null && nodeId2 == null) { + nodeIdComparison = 0; + } else if (nodeId1 == null) { + nodeIdComparison = 1; + } else if (nodeId2 == null) { + nodeIdComparison = -1; + } else { + nodeIdComparison = -nodeId1.compareTo(nodeId2); + } + + if (nodeIdComparison != 0) { + return nodeIdComparison; + } + + return -Long.compare(o1.getEventId(), o2.getEventId()); + } + }); + + final int maxResults = request.getMaxResults().intValue(); + final List selectedResults; + if (allResults.size() < maxResults) { + selectedResults = allResults; + } else { + selectedResults = allResults.subList(0, maxResults); + } + + // include any errors + if (errors.size() > 0) { + results.setErrors(errors); + } + + results.setTotalCount(totalRecords); + results.setTotal(FormatUtils.formatCount(totalRecords)); + results.setProvenanceEvents(selectedResults); + results.setOldestEvent(oldestEventDate); + results.setGenerated(new Date()); + provenanceDto.setPercentCompleted(percentageComplete); + provenanceDto.setFinished(finished); + } + + private void mergeRemoteProcessGroup(final RemoteProcessGroupDTO remoteProcessGroup, final Map remoteProcessGroupMap) { + final RemoteProcessGroupContentsDTO remoteProcessGroupContents = remoteProcessGroup.getContents(); + + Boolean mergedIsTargetSecure = null; + final List mergedAuthorizationIssues = new ArrayList<>(); + final Set mergedInputPorts = new HashSet<>(); + final Set mergedOutputPorts = new HashSet<>(); + + for (final Map.Entry nodeEntry : remoteProcessGroupMap.entrySet()) { + final NodeIdentifier nodeId = nodeEntry.getKey(); + final RemoteProcessGroupDTO nodeRemoteProcessGroupDto = nodeEntry.getValue(); + + // merge the issues + final List nodeAuthorizationIssues = nodeRemoteProcessGroupDto.getAuthorizationIssues(); + if (nodeAuthorizationIssues != null && !nodeAuthorizationIssues.isEmpty()) { + for (final String nodeAuthorizationIssue : nodeAuthorizationIssues) { + mergedAuthorizationIssues.add(nodeId.getApiAddress() + ":" + nodeId.getApiPort() + " -- " + nodeAuthorizationIssue); + } + } + + // use the first target secure flag since they will all be the same + final Boolean nodeIsTargetSecure = nodeRemoteProcessGroupDto.isTargetSecure(); + if (mergedIsTargetSecure == null) { + mergedIsTargetSecure = nodeIsTargetSecure; + } + + // merge the ports in the contents + final RemoteProcessGroupContentsDTO nodeRemoteProcessGroupContentsDto = nodeRemoteProcessGroupDto.getContents(); + if (remoteProcessGroupContents != null && nodeRemoteProcessGroupContentsDto != null) { + if (nodeRemoteProcessGroupContentsDto.getInputPorts() != null) { + mergedInputPorts.addAll(nodeRemoteProcessGroupContentsDto.getInputPorts()); + } + if (nodeRemoteProcessGroupContentsDto.getOutputPorts() != null) { + mergedOutputPorts.addAll(nodeRemoteProcessGroupContentsDto.getOutputPorts()); + } + } + } + + if (remoteProcessGroupContents != null) { + if (!mergedInputPorts.isEmpty()) { + remoteProcessGroupContents.setInputPorts(mergedInputPorts); + } + if (!mergedOutputPorts.isEmpty()) { + remoteProcessGroupContents.setOutputPorts(mergedOutputPorts); + } + } + + if (mergedIsTargetSecure != null) { + remoteProcessGroup.setTargetSecure(mergedIsTargetSecure); + } + + if (!mergedAuthorizationIssues.isEmpty()) { + remoteProcessGroup.setAuthorizationIssues(mergedAuthorizationIssues); + } + } + + // requires write lock to be already acquired unless request is not mutable + private NodeResponse mergeResponses(final URI uri, final String method, final Set nodeResponses, final boolean mutableRequest) { + // holds the one response of all the node responses to return to the client + NodeResponse clientResponse = null; + + // holds the set of node responses that did not result in a 2XX response + final Set problematicNodeResponses = new HashSet<>(); + + // map updated node to its response + final Map updatedNodesMap = new HashMap<>(); + for (final Map.Entry entry : httpResponseMapper.map(uri, nodeResponses).entrySet()) { + + final NodeResponse nodeResponse = entry.getKey(); + final Status nodeStatus = entry.getValue(); + + // create new "updated" node by cloning old node and updating status + final Node currentNode = getRawNode(nodeResponse.getNodeId().getId()); + final Node updatedNode = currentNode.clone(); + updatedNode.setStatus(nodeStatus); + + // map updated node to its response + updatedNodesMap.put(updatedNode, nodeResponse); + + // record a client request and any requests that resulted in disconnection + if (nodeStatus == Status.CONNECTED) { + clientResponse = nodeResponse; + } else if (nodeStatus == Status.DISCONNECTED) { + problematicNodeResponses.add(nodeResponse); + } + } + + // determine if we have at least one response + final boolean hasClientResponse = clientResponse != null; + final boolean hasSuccessfulClientResponse = hasClientResponse && clientResponse.is2xx(); + + // drain the responses from the socket for those responses not being sent to the client + final Set nodeResponsesToDrain = new HashSet<>(updatedNodesMap.values()); + nodeResponsesToDrain.remove(clientResponse); + + if (hasSuccessfulClientResponse && isProcessorEndpoint(uri, method)) { + final ProcessorEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessorEntity.class); + final ProcessorDTO processor = responseEntity.getProcessor(); + + final Map processorMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ProcessorEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessorEntity.class); + final ProcessorDTO nodeProcessor = nodeResponseEntity.getProcessor(); + processorMap.put(nodeResponse.getNodeId(), nodeProcessor); + } + + mergeProcessorValidationErrors(processor, processorMap); + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isProcessorsEndpoint(uri, method)) { + final ProcessorsEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessorsEntity.class); + final Set processors = responseEntity.getProcessors(); + + final Map> processorMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ProcessorsEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessorsEntity.class); + final Set nodeProcessors = nodeResponseEntity.getProcessors(); + + for (final ProcessorDTO nodeProcessor : nodeProcessors) { + Map innerMap = processorMap.get(nodeProcessor.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + processorMap.put(nodeProcessor.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeProcessor); + } + } + + for (final ProcessorDTO processor : processors) { + final String procId = processor.getId(); + final Map mergeMap = processorMap.get(procId); + + mergeProcessorValidationErrors(processor, mergeMap); + } + + // create a new client response + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isProcessGroupEndpoint(uri, method)) { + final ProcessGroupEntity responseEntity = clientResponse.getClientResponse().getEntity(ProcessGroupEntity.class); + final ProcessGroupDTO responseDto = responseEntity.getProcessGroup(); + + final FlowSnippetDTO contents = responseDto.getContents(); + if (contents == null) { + if (!nodeResponsesToDrain.isEmpty()) { + drainResponses(nodeResponsesToDrain); + } + } else { + final Map> processorMap = new HashMap<>(); + final Map> remoteProcessGroupMap = new HashMap<>(); + + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ProcessGroupEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ProcessGroupEntity.class); + final ProcessGroupDTO nodeProcessGroup = nodeResponseEntity.getProcessGroup(); + + for (final ProcessorDTO nodeProcessor : nodeProcessGroup.getContents().getProcessors()) { + Map innerMap = processorMap.get(nodeProcessor.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + processorMap.put(nodeProcessor.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeProcessor); + } + + for (final RemoteProcessGroupDTO nodeRemoteProcessGroup : nodeProcessGroup.getContents().getRemoteProcessGroups()) { + Map innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup); + } + } + + for (final ProcessorDTO processor : contents.getProcessors()) { + final String procId = processor.getId(); + final Map mergeMap = processorMap.get(procId); + + mergeProcessorValidationErrors(processor, mergeMap); + } + + for (final RemoteProcessGroupDTO remoteProcessGroup : contents.getRemoteProcessGroups()) { + if (remoteProcessGroup.getContents() != null) { + final String remoteProcessGroupId = remoteProcessGroup.getId(); + final Map mergeMap = remoteProcessGroupMap.get(remoteProcessGroupId); + + mergeRemoteProcessGroup(remoteProcessGroup, mergeMap); + } + } + } + + // create a new client response + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && (isTemplateEndpoint(uri, method) || isFlowSnippetEndpoint(uri, method))) { + final FlowSnippetEntity responseEntity = clientResponse.getClientResponse().getEntity(FlowSnippetEntity.class); + final FlowSnippetDTO contents = responseEntity.getContents(); + + if (contents == null) { + if (!nodeResponsesToDrain.isEmpty()) { + drainResponses(nodeResponsesToDrain); + } + } else { + final Map> processorMap = new HashMap<>(); + final Map> remoteProcessGroupMap = new HashMap<>(); + + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final FlowSnippetEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(FlowSnippetEntity.class); + final FlowSnippetDTO nodeContents = nodeResponseEntity.getContents(); + + for (final ProcessorDTO nodeProcessor : nodeContents.getProcessors()) { + Map innerMap = processorMap.get(nodeProcessor.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + processorMap.put(nodeProcessor.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeProcessor); + } + + for (final RemoteProcessGroupDTO nodeRemoteProcessGroup : nodeContents.getRemoteProcessGroups()) { + Map innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup); + } + } + + for (final ProcessorDTO processor : contents.getProcessors()) { + final String procId = processor.getId(); + final Map mergeMap = processorMap.get(procId); + + mergeProcessorValidationErrors(processor, mergeMap); + } + + for (final RemoteProcessGroupDTO remoteProcessGroup : contents.getRemoteProcessGroups()) { + if (remoteProcessGroup.getContents() != null) { + final String remoteProcessGroupId = remoteProcessGroup.getId(); + final Map mergeMap = remoteProcessGroupMap.get(remoteProcessGroupId); + + mergeRemoteProcessGroup(remoteProcessGroup, mergeMap); + } + } + } + + // create a new client response + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && (isRemoteProcessGroupEndpoint(uri, method))) { + final RemoteProcessGroupEntity responseEntity = clientResponse.getClientResponse().getEntity(RemoteProcessGroupEntity.class); + final RemoteProcessGroupDTO remoteProcessGroup = responseEntity.getRemoteProcessGroup(); + + final Map remoteProcessGroupMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final RemoteProcessGroupEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(RemoteProcessGroupEntity.class); + final RemoteProcessGroupDTO nodeRemoteProcessGroup = nodeResponseEntity.getRemoteProcessGroup(); + + remoteProcessGroupMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup); + } + mergeRemoteProcessGroup(remoteProcessGroup, remoteProcessGroupMap); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && (isRemoteProcessGroupsEndpoint(uri, method))) { + final RemoteProcessGroupsEntity responseEntity = clientResponse.getClientResponse().getEntity(RemoteProcessGroupsEntity.class); + final Set remoteProcessGroups = responseEntity.getRemoteProcessGroups(); + + final Map> remoteProcessGroupMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final RemoteProcessGroupsEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(RemoteProcessGroupsEntity.class); + final Set nodeRemoteProcessGroups = nodeResponseEntity.getRemoteProcessGroups(); + + for (final RemoteProcessGroupDTO nodeRemoteProcessGroup : nodeRemoteProcessGroups) { + Map innerMap = remoteProcessGroupMap.get(nodeRemoteProcessGroup.getId()); + if (innerMap == null) { + innerMap = new HashMap<>(); + remoteProcessGroupMap.put(nodeRemoteProcessGroup.getId(), innerMap); + } + + innerMap.put(nodeResponse.getNodeId(), nodeRemoteProcessGroup); + } + } + + for (final RemoteProcessGroupDTO remoteProcessGroup : remoteProcessGroups) { + final String remoteProcessGroupId = remoteProcessGroup.getId(); + final Map mergeMap = remoteProcessGroupMap.get(remoteProcessGroupId); + + mergeRemoteProcessGroup(remoteProcessGroup, mergeMap); + } + + // create a new client response + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isProvenanceQueryEndpoint(uri, method)) { + final ProvenanceEntity responseEntity = clientResponse.getClientResponse().getEntity(ProvenanceEntity.class); + final ProvenanceDTO query = responseEntity.getProvenance(); + + final Map resultsMap = new HashMap<>(); + for (final NodeResponse nodeResponse : updatedNodesMap.values()) { + if (problematicNodeResponses.contains(nodeResponse)) { + continue; + } + + final ProvenanceEntity nodeResponseEntity = (nodeResponse == clientResponse) ? responseEntity : nodeResponse.getClientResponse().getEntity(ProvenanceEntity.class); + final ProvenanceDTO nodeQuery = nodeResponseEntity.getProvenance(); + + resultsMap.put(nodeResponse.getNodeId(), nodeQuery); + } + mergeProvenanceQueryResults(query, resultsMap, problematicNodeResponses); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else if (hasSuccessfulClientResponse && isProvenanceEventEndpoint(uri, method)) { + final ProvenanceEventEntity responseEntity = clientResponse.getClientResponse().getEntity(ProvenanceEventEntity.class); + final ProvenanceEventDTO event = responseEntity.getProvenanceEvent(); + + // this request was sent to a specific node... populate its details + final NodeIdentifier nodeId = clientResponse.getNodeId(); + event.setClusterNodeId(nodeId.getId()); + event.setClusterNodeAddress(nodeId.getApiAddress() + ":" + nodeId.getApiPort()); + + clientResponse = new NodeResponse(clientResponse, responseEntity); + } else { + if (!nodeResponsesToDrain.isEmpty()) { + drainResponses(nodeResponsesToDrain); + } + } + + /* + * Nodes that encountered issues handling the request are marked as + * disconnected for mutable requests (e.g., post, put, delete). For + * other requests (e.g., get, head), the nodes remain in their current + * state even if they had problems handling the request. + */ + if (mutableRequest) { + + // set the updated nodes + nodes.removeAll(updatedNodesMap.keySet()); + nodes.addAll(updatedNodesMap.keySet()); + + // notify service of updated node set + notifyDataFlowManagementServiceOfNodeStatusChange(); + + // mark flow as stale since this request could have changed the flow + notifyDataFlowManagmentServiceOfFlowStateChange(PersistedFlowState.STALE); + + // disconnect problematic nodes + if (!problematicNodeResponses.isEmpty()) { + if (problematicNodeResponses.size() < nodeResponses.size()) { + logger.warn(String.format("One or more nodes failed to process URI '%s'. Requesting each node to disconnect from cluster.", uri)); + disconnectNodes(problematicNodeResponses, "Failed to process URI " + uri); + } else { + logger.warn("All nodes failed to process URI {}. As a result, no node will be disconnected from cluster", uri); + } + } + + // if at least one node satisfied the request, then audit the action + if (hasClientResponse) { + try { + // get the cluster context from the response header + final String serializedClusterContext = clientResponse.getClientResponse().getHeaders().getFirst(CLUSTER_CONTEXT_HTTP_HEADER); + if (StringUtils.isNotBlank(serializedClusterContext)) { + // deserialize object + final Serializable clusterContextObj = WebUtils.deserializeHexToObject(serializedClusterContext); + + // if we have a valid object, audit the actions + if (clusterContextObj instanceof ClusterContext) { + final ClusterContext clusterContext = (ClusterContext) clusterContextObj; + if (auditService != null) { + try { + auditService.addActions(clusterContext.getActions()); + } catch (Throwable t) { + logger.warn("Unable to record actions: " + t.getMessage()); + if (logger.isDebugEnabled()) { + logger.warn(StringUtils.EMPTY, t); + } + } + } + revision = clusterContext.getRevision(); + } + } + } catch (final ClassNotFoundException cnfe) { + logger.warn("Classpath issue detected because failed to deserialize cluster context from node response due to: " + cnfe, cnfe); + } + } + } + + return clientResponse; + } + + /** + * Drains the node responses off of the socket to ensure that the socket is + * appropriately cleaned-up. + * + * @param nodeResponses the collection of node responses + */ + private void drainResponses(final Collection nodeResponses) { + // fail fast if nothing to do + if (nodeResponses.isEmpty()) { + return; + } + + final ExecutorService executorService = Executors.newFixedThreadPool(properties.getClusterManagerProtocolThreads()); + final CompletionService completionService = new ExecutorCompletionService<>(executorService); + for (final NodeResponse nodeResponse : nodeResponses) { + // if we received a response, then clear out the response data + if (!nodeResponse.hasThrowable()) { + completionService.submit(new Runnable() { + @Override + public void run() { + try { + ((StreamingOutput) nodeResponse.getResponse().getEntity()).write( + new OutputStream() { + @Override + public void write(final int b) { /* drain response */ } + } + ); + } catch (final IOException | WebApplicationException ex) { + logger.info("Failed clearing out non-client response buffer due to: " + ex, ex); + } + } + }, null); + } + } + + executorService.shutdown(); + } + + /** + * A helper method to disconnect nodes that returned unsuccessful HTTP + * responses because of a replicated request. Disconnection requests are + * sent concurrently. + * + * @param nodeResponses + */ + private void disconnectNodes(final Set nodeResponses, final String explanation) { + // return fast if nothing to do + if (nodeResponses == null || nodeResponses.isEmpty()) { + return; + } + + final ExecutorService executorService = Executors.newFixedThreadPool(properties.getClusterManagerProtocolThreads()); + final CompletionService completionService = new ExecutorCompletionService<>(executorService); + for (final NodeResponse nodeResponse : nodeResponses) { + completionService.submit(new Runnable() { + @Override + public void run() { + final NodeIdentifier nodeId = nodeResponse.getNodeId(); + final int responseStatus = nodeResponse.getStatus(); + final URI requestUri = nodeResponse.getRequestUri(); + final StringBuilder msgBuilder = new StringBuilder(); + msgBuilder + .append("Requesting disconnection for node ") + .append(nodeId) + .append(" for request URI ") + .append(requestUri); + if (nodeResponse.hasThrowable()) { + msgBuilder.append(" because manager encountered exception when issuing request: ") + .append(nodeResponse.getThrowable()); + // log stack trace anytime we have a throwable + ((NiFiLog) logger).getWrappedLog().info(msgBuilder.toString(), nodeResponse.getThrowable()); + addEvent(nodeId, "Manager encountered exception when issuing request for URI " + requestUri); + addBulletin(nodeId, Severity.ERROR, "Manager encountered exception when issuing request for URI " + requestUri + "; node will be disconnected"); + } else { + msgBuilder.append(" because HTTP response status was ") + .append(responseStatus); + logger.info(msgBuilder.toString()); + addEvent(nodeId, "HTTP response status was unsuccessful (" + responseStatus + ") for request URI " + requestUri); + addBulletin(nodeId, Severity.ERROR, "HTTP response status was unsuccessful (" + responseStatus + ") for request URI " + requestUri); + } + requestDisconnectionQuietly(nodeId, explanation); + } + }, null); + } + + executorService.shutdown(); + } + + /** + * Returns false if an internal protocol message was received by a node + * listed in the firewall. If no firewall is configured, then false is + * always returned. + * + * @param ip the IP of the remote machine + * + * @return false if the IP is listed in the firewall or if the firewall is + * not configured; true otherwise + */ + private boolean isBlockedByFirewall(final String ip) { + if (isFirewallConfigured()) { + return !clusterFirewall.isPermissible(ip); + } else { + return false; + } + } + + private Set getRawNodes(final Status... statuses) { + readLock.lock(); + try { + final Set result = new HashSet<>(); + if (statuses == null || statuses.length == 0) { + result.addAll(nodes); + } else { + for (final Node node : nodes) { + for (final Node.Status status : statuses) { + if (node.getStatus() == status) { + result.add(node); + break; + } + } + } + } + return result; + } finally { + readLock.unlock("getRawNodes(Status...)"); + } + } + + private Node getRawNode(final String nodeId) { + readLock.lock(); + try { + for (final Node node : nodes) { + if (node.getNodeId().getId().equals(nodeId)) { + return node; + } + } + return null; + } finally { + readLock.unlock("getRawNode(String)"); + } + } + + /** + * Resolves a proposed node identifier to a node identifier that the manager + * approves. If the proposed node identifier conflicts with an existing node + * identifier, then an approved node identifier is generated and returned to + * the caller. + * + * @param proposedNodeId a proposed identifier + * + * @return the node identifier that should be used + */ + private NodeIdentifier resolveProposedNodeIdentifier(final NodeIdentifier proposedNodeId) { + readLock.lock(); + try { + for (final Node node : nodes) { + final NodeIdentifier nodeId = node.getNodeId(); + + // are the ids the same + final boolean sameId = nodeId.equals(proposedNodeId); + + // are the service coordinates the same + final boolean sameServiceCoordinates = nodeId.logicallyEquals(proposedNodeId); + + if (sameId && sameServiceCoordinates) { + // we know about this node and it has the same ID, so the proposal is fine + return proposedNodeId; + } else if (sameId && !sameServiceCoordinates) { + // proposed ID conflicts with existing node ID, so assign a new ID + final NodeIdentifier resolvedIdentifier = new NodeIdentifier( + UUID.randomUUID().toString(), + proposedNodeId.getApiAddress(), + proposedNodeId.getApiPort(), + proposedNodeId.getSocketAddress(), + proposedNodeId.getSocketPort()); + logger.info(String.format("Using Node Identifier %s because proposed node identifier %s conflicts existing node identifiers", + resolvedIdentifier, proposedNodeId)); + return resolvedIdentifier; + } else if (!sameId && sameServiceCoordinates) { + // we know about this node, so we'll use the existing ID + logger.debug(String.format("Using Node Identifier %s because proposed node identifier %s matches the service coordinates", + nodeId, proposedNodeId)); + return nodeId; + } + + } + + // proposal does not conflict with existing nodes + return proposedNodeId; + } finally { + readLock.unlock("resolveProposedNodeIdentifier"); + } + } + + private boolean isHeartbeatMonitorRunning() { + readLock.lock(); + try { + return heartbeatMonitor != null; + } finally { + readLock.unlock("isHeartbeatMonitorRunning"); + } + } + + private boolean canChangeNodeState(final String method, final URI uri) { + return (HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.POST.equalsIgnoreCase(method) || HttpMethod.PUT.equalsIgnoreCase(method)); + } + + private void notifyDataFlowManagementServiceOfNodeStatusChange() { + writeLock.lock(); + try { + // tell service about the currently connected nodes + logger.debug("Notifying DataFlow Management Service of current set of connected nodes."); + dataFlowManagementService.setNodeIds(getNodeIds(Status.CONNECTED)); + } finally { + writeLock.unlock("notifyDataFlowManagementServiceOfNodeStatusChange"); + } + } + + private void notifyDataFlowManagmentServiceOfFlowStateChange(final PersistedFlowState newState) { + writeLock.lock(); + try { + logger.debug("Notifying DataFlow Management Service that flow state is " + newState); + dataFlowManagementService.setPersistedFlowState(newState); + if (newState != PersistedFlowState.CURRENT) { + cachedDataFlow = null; + /* do not reset primary node ID because only the data flow has changed */ + } + } finally { + writeLock.unlock("notifyDataFlowManagementServiceOfFlowStateChange"); + } + } + + private void logNodes(final String header, final Logger logger) { + if (logger.isTraceEnabled()) { + if (StringUtils.isNotBlank(header)) { + logger.trace(header); + } + for (final Node node : getNodes()) { + logger.trace(node.getNodeId() + " : " + node.getStatus()); + } + } + } + + private void executeSafeModeTask() { + + new Thread(new Runnable() { + + private final long threadStartTime = System.currentTimeMillis(); + + @Override + public void run() { + logger.info("Entering safe mode..."); + final int safeModeSeconds = (int) FormatUtils.getTimeDuration(properties.getClusterManagerSafeModeDuration(), TimeUnit.SECONDS); + final long timeToElect = (safeModeSeconds <= 0) ? Long.MAX_VALUE : threadStartTime + TimeUnit.MILLISECONDS.convert(safeModeSeconds, TimeUnit.SECONDS); + boolean exitSafeMode = false; + while (isRunning()) { + + writeLock.lock(); + try { + + final long currentTime = System.currentTimeMillis(); + if (timeToElect < currentTime) { + final Set connectedNodeIds = getNodeIds(Status.CONNECTED); + if (!connectedNodeIds.isEmpty()) { + // get first connected node ID + final NodeIdentifier connectedNodeId = connectedNodeIds.iterator().next(); + if (assignPrimaryRole(connectedNodeId)) { + try { + setPrimaryNodeId(connectedNodeId); + exitSafeMode = true; + } catch (final DaoException de) { + final String message = String.format("Failed to persist primary node ID '%s' in cluster dataflow.", connectedNodeId); + logger.warn(message); + addBulletin(connectedNodeId, Severity.WARNING, message); + revokePrimaryRole(connectedNodeId); + } + } + } + } + + if (!isInSafeMode()) { + // a primary node has been selected outside of this thread + exitSafeMode = true; + logger.info("Exiting safe mode because " + primaryNodeId + " has been assigned the primary role."); + break; + } + } finally { + writeLock.unlock("executeSafeModeTask"); + } + + if (!exitSafeMode) { + // sleep for a bit + try { + Thread.sleep(1000); + } catch (final InterruptedException ie) { + return; + } + } + + } + } + }).start(); + } + + /** + * This timer task simply processes any pending heartbeats. This timer task + * is not strictly needed, as HeartbeatMonitoringTimerTask will do this. + * However, this task is scheduled much more frequently and by processing + * the heartbeats more frequently, the stats that we report have less of a + * delay. + */ + private class ProcessPendingHeartbeatsTask extends TimerTask { + + @Override + public void run() { + writeLock.lock(); + try { + processPendingHeartbeats(); + } finally { + writeLock.unlock("Process Pending Heartbeats Task"); + } + } + } + + /** + * A timer task to detect nodes that have not sent a heartbeat in a while. + * The "problem" nodes are marked as disconnected due to lack of heartbeat + * by the task. No disconnection request is sent to the node. This is + * because either the node is not functioning in which case sending the + * request is futile or the node is running a bit slow. In the latter case, + * we'll wait for the next heartbeat and send a reconnection request when we + * process the heartbeat in the heartbeatHandler() method. + */ + private class HeartbeatMonitoringTimerTask extends TimerTask { + + @Override + public void run() { + // keep track of any status changes + boolean statusChanged = false; + + writeLock.lock(); + try { + // process all of the heartbeats before we decided to kick anyone out of the cluster. + logger.debug("Processing pending heartbeats..."); + processPendingHeartbeats(); + + logger.debug("Executing heartbeat monitoring"); + + // check for any nodes that have not heartbeated in a long time + for (final Node node : getRawNodes(Status.CONNECTED)) { + // return prematurely if we were interrupted + if (Thread.currentThread().isInterrupted()) { + return; + } + + // check if we received a recent heartbeat, changing status to disconnected if necessary + final long lastHeardTimestamp = node.getHeartbeat().getCreatedTimestamp(); + final int heartbeatGapSeconds = (int) (new Date().getTime() - lastHeardTimestamp) / 1000; + if (heartbeatGapSeconds > getMaxHeartbeatGapSeconds()) { + node.setHeartbeatDisconnection(); + addEvent(node.getNodeId(), "Node disconnected due to lack of heartbeat."); + addBulletin(node, Severity.WARNING, "Node disconnected due to lack of heartbeat"); + statusChanged = true; + } + } + + // if a status change occurred, make the necessary updates + if (statusChanged) { + logNodes("Heartbeat Monitoring disconnected node(s)", logger); + // notify service of updated node set + notifyDataFlowManagementServiceOfNodeStatusChange(); + } else { + logNodes("Heartbeat Monitoring determined all nodes are healthy", logger); + } + } catch (final Exception ex) { + logger.warn("Heartbeat monitor experienced exception while monitoring: " + ex, ex); + } finally { + writeLock.unlock("HeartbeatMonitoringTimerTask"); + } + } + } + + @Override + public ClusterNodeInformation getNodeInformation() { + readLock.lock(); + try { + final Collection nodeInfos = new ArrayList<>(); + for (final Node node : getRawNodes(Status.CONNECTED)) { + final NodeIdentifier id = node.getNodeId(); + final HeartbeatPayload heartbeat = node.getHeartbeatPayload(); + if (heartbeat == null) { + continue; + } + + final Integer siteToSitePort = heartbeat.getSiteToSitePort(); + if (siteToSitePort == null) { + continue; + } + final int flowFileCount = (int) heartbeat.getTotalFlowFileCount(); + final NodeInformation nodeInfo = new NodeInformation(id.getApiAddress(), siteToSitePort, id.getApiPort(), + heartbeat.isSiteToSiteSecure(), flowFileCount); + nodeInfos.add(nodeInfo); + } + + final ClusterNodeInformation clusterNodeInfo = new ClusterNodeInformation(); + clusterNodeInfo.setNodeInformation(nodeInfos); + return clusterNodeInfo; + } finally { + readLock.unlock("getNodeInformation"); + } + } + + @Override + public BulletinRepository getBulletinRepository() { + return bulletinRepository; + } + + @Override + public ProcessGroupStatus getProcessGroupStatus(final String groupId) { + final Set connectedNodes = getNodes(Node.Status.CONNECTED); + + // ensure there are some nodes in the cluster + if (connectedNodes.isEmpty()) { + throw new NoConnectedNodesException(); + } + + ProcessGroupStatus mergedProcessGroupStatus = null; + for (final Node node : connectedNodes) { + final NodeIdentifier nodeId = node.getNodeId(); + final HeartbeatPayload nodeHeartbeatPayload = node.getHeartbeatPayload(); + if (nodeHeartbeatPayload == null) { + continue; + } + final ProcessGroupStatus nodeRootProcessGroupStatus = nodeHeartbeatPayload.getProcessGroupStatus(); + final ProcessGroupStatus nodeProcessGroupStatus = groupId.equals(ROOT_GROUP_ID_ALIAS) ? nodeRootProcessGroupStatus : getProcessGroupStatus(nodeRootProcessGroupStatus, groupId); + if (nodeProcessGroupStatus == null) { + continue; + } + + if (mergedProcessGroupStatus == null) { + mergedProcessGroupStatus = nodeProcessGroupStatus.clone(); + + // update any issues with the node label + if (mergedProcessGroupStatus.getRemoteProcessGroupStatus() != null) { + for (final RemoteProcessGroupStatus remoteProcessGroupStatus : mergedProcessGroupStatus.getRemoteProcessGroupStatus()) { + final List nodeAuthorizationIssues = remoteProcessGroupStatus.getAuthorizationIssues(); + if (!nodeAuthorizationIssues.isEmpty()) { + for (final ListIterator iter = nodeAuthorizationIssues.listIterator(); iter.hasNext();) { + final String Issue = iter.next(); + iter.set("[" + nodeId.getApiAddress() + ":" + nodeId.getApiPort() + "] -- " + Issue); + } + remoteProcessGroupStatus.setAuthorizationIssues(nodeAuthorizationIssues); + } + } + } + } else { + final ProcessGroupStatus nodeClone = nodeProcessGroupStatus.clone(); + for (final RemoteProcessGroupStatus remoteProcessGroupStatus : nodeClone.getRemoteProcessGroupStatus()) { + final List nodeAuthorizationIssues = remoteProcessGroupStatus.getAuthorizationIssues(); + if (!nodeAuthorizationIssues.isEmpty()) { + for (final ListIterator iter = nodeAuthorizationIssues.listIterator(); iter.hasNext();) { + final String Issue = iter.next(); + iter.set("[" + nodeId.getApiAddress() + ":" + nodeId.getApiPort() + "] -- " + Issue); + } + remoteProcessGroupStatus.setAuthorizationIssues(nodeAuthorizationIssues); + } + } + + ProcessGroupStatus.merge(mergedProcessGroupStatus, nodeClone); + } + } + + return mergedProcessGroupStatus; + } + + private ProcessGroupStatus getProcessGroupStatus(final ProcessGroupStatus parent, final String groupId) { + if (parent.getId().equals(groupId)) { + return parent; + } + + for (final ProcessGroupStatus child : parent.getProcessGroupStatus()) { + final ProcessGroupStatus matching = getProcessGroupStatus(child, groupId); + if (matching != null) { + return matching; + } + } + + return null; + } + + @Override + public SystemDiagnostics getSystemDiagnostics() { + final Set connectedNodes = getNodes(Node.Status.CONNECTED); + + // ensure there are some nodes... + if (connectedNodes.isEmpty()) { + throw new NoConnectedNodesException(); + } + + SystemDiagnostics clusterDiagnostics = null; + for (final Node node : connectedNodes) { + final HeartbeatPayload nodeHeartbeatPayload = node.getHeartbeatPayload(); + if (nodeHeartbeatPayload == null) { + continue; + } + final SystemDiagnostics nodeDiagnostics = nodeHeartbeatPayload.getSystemDiagnostics(); + if (nodeDiagnostics == null) { + continue; + } + + if (clusterDiagnostics == null) { + clusterDiagnostics = nodeDiagnostics.clone(); + } else { + merge(clusterDiagnostics, nodeDiagnostics); + } + } + + return clusterDiagnostics; + } + + private void merge(final SystemDiagnostics target, final SystemDiagnostics sd) { + + // threads + target.setDaemonThreads(target.getDaemonThreads() + sd.getDaemonThreads()); + target.setTotalThreads(target.getTotalThreads() + sd.getTotalThreads()); + + // heap + target.setTotalHeap(target.getTotalHeap() + sd.getTotalHeap()); + target.setUsedHeap(target.getUsedHeap() + sd.getUsedHeap()); + target.setMaxHeap(target.getMaxHeap() + sd.getMaxHeap()); + + // non heap + target.setTotalNonHeap(target.getTotalNonHeap() + sd.getTotalNonHeap()); + target.setUsedNonHeap(target.getUsedNonHeap() + sd.getUsedNonHeap()); + target.setMaxNonHeap(target.getMaxNonHeap() + sd.getMaxNonHeap()); + + // processors + target.setAvailableProcessors(target.getAvailableProcessors() + sd.getAvailableProcessors()); + + // load + if (sd.getProcessorLoadAverage() != null) { + if (target.getProcessorLoadAverage() != null) { + target.setProcessorLoadAverage(target.getProcessorLoadAverage() + sd.getProcessorLoadAverage()); + } else { + target.setProcessorLoadAverage(sd.getProcessorLoadAverage()); + } + } + + // db disk usage + merge(target.getFlowFileRepositoryStorageUsage(), sd.getFlowFileRepositoryStorageUsage()); + + // repo disk usage + final Map targetContentRepoMap; + if (target.getContentRepositoryStorageUsage() == null) { + targetContentRepoMap = new LinkedHashMap<>(); + target.setContentRepositoryStorageUsage(targetContentRepoMap); + } else { + targetContentRepoMap = target.getContentRepositoryStorageUsage(); + } + if (sd.getContentRepositoryStorageUsage() != null) { + for (final Map.Entry sdEntry : sd.getContentRepositoryStorageUsage().entrySet()) { + final StorageUsage mergedDiskUsage = targetContentRepoMap.get(sdEntry.getKey()); + if (mergedDiskUsage == null) { + targetContentRepoMap.put(sdEntry.getKey(), sdEntry.getValue()); + } else { + merge(mergedDiskUsage, sdEntry.getValue()); + } + } + } + + // garbage collection + final Map targetGarbageCollection; + if (target.getGarbageCollection() == null) { + targetGarbageCollection = new LinkedHashMap<>(); + target.setGarbageCollection(targetGarbageCollection); + } else { + targetGarbageCollection = target.getGarbageCollection(); + } + if (sd.getGarbageCollection() != null) { + for (final Map.Entry gcEntry : sd.getGarbageCollection().entrySet()) { + final GarbageCollection mergedGarbageCollection = targetGarbageCollection.get(gcEntry.getKey()); + if (mergedGarbageCollection == null) { + targetGarbageCollection.put(gcEntry.getKey(), gcEntry.getValue().clone()); + } else { + merge(mergedGarbageCollection, gcEntry.getValue()); + } + } + } + } + + private void merge(final StorageUsage target, final StorageUsage du) { + target.setFreeSpace(target.getFreeSpace() + du.getFreeSpace()); + target.setTotalSpace(target.getTotalSpace() + du.getTotalSpace()); + } + + private void merge(final GarbageCollection target, final GarbageCollection gc) { + target.setCollectionCount(target.getCollectionCount() + gc.getCollectionCount()); + target.setCollectionTime(target.getCollectionTime() + gc.getCollectionTime()); + } + + public static Date normalizeStatusSnapshotDate(final Date toNormalize, final long numMillis) { + final long time = toNormalize.getTime(); + return new Date(time - (time % numMillis)); + } + + private NodeDTO createNodeDTO(final Node node) { + final NodeDTO nodeDto = new NodeDTO(); + final NodeIdentifier nodeId = node.getNodeId(); + nodeDto.setNodeId(nodeId.getId()); + nodeDto.setAddress(nodeId.getApiAddress()); + nodeDto.setApiPort(nodeId.getApiPort()); + nodeDto.setStatus(node.getStatus().name()); + nodeDto.setPrimary(node.equals(getPrimaryNode())); + final Date connectionRequested = new Date(node.getConnectionRequestedTimestamp()); + nodeDto.setConnectionRequested(connectionRequested); + + return nodeDto; + } + + private List aggregate(Map> snapshotsToAggregate) { + // Aggregate the snapshots + final List aggregatedSnapshotDtos = new ArrayList<>(); + for (final Map.Entry> entry : snapshotsToAggregate.entrySet()) { + final StatusSnapshotDTO dto = new StatusSnapshotDTO(); + dto.setTimestamp(entry.getKey()); + + final List snapshots = entry.getValue(); + final StatusSnapshot reducedSnapshot = snapshots.get(0).getValueReducer().reduce(snapshots); + dto.setStatusMetrics(StatusHistoryUtil.createStatusSnapshotDto(reducedSnapshot).getStatusMetrics()); + + aggregatedSnapshotDtos.add(dto); + } + + return aggregatedSnapshotDtos; + } + + public ClusterStatusHistoryDTO getProcessorStatusHistory(final String processorId) { + return getProcessorStatusHistory(processorId, null, null, Integer.MAX_VALUE); + } + + public ClusterStatusHistoryDTO getProcessorStatusHistory(final String processorId, final Date startDate, final Date endDate, final int preferredDataPoints) { + final List nodeHistories = new ArrayList<>(); + + StatusHistoryDTO lastStatusHistory = null; + Set> processorDescriptors = new LinkedHashSet<>(); + Map> snapshotsToAggregate = new TreeMap<>(); + + for (final Node node : getRawNodes()) { + final ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); + if (statusRepository == null) { + continue; + } + + final StatusHistory statusHistory = statusRepository.getProcessorStatusHistory(processorId, startDate, endDate, preferredDataPoints); + if (statusHistory == null) { + continue; + } + + processorDescriptors.addAll(statusRepository.getProcessorMetricDescriptors()); + + // record the status history (last) to get the component details for use later + final StatusHistoryDTO statusHistoryDto = createStatusHistoryDto(statusHistory); + lastStatusHistory = statusHistoryDto; + + final NodeStatusHistoryDTO nodeHistory = new NodeStatusHistoryDTO(); + nodeHistory.setStatusHistory(statusHistoryDto); + nodeHistory.setNode(createNodeDTO(node)); + nodeHistories.add(nodeHistory); + + // collect all of the snapshots to aggregate + for (final StatusSnapshot snapshot : statusHistory.getStatusSnapshots()) { + final Date normalizedDate = normalizeStatusSnapshotDate(snapshot.getTimestamp(), componentStatusSnapshotMillis); + List snapshots = snapshotsToAggregate.get(normalizedDate); + if (snapshots == null) { + snapshots = new ArrayList<>(); + snapshotsToAggregate.put(normalizedDate, snapshots); + } + snapshots.add(snapshot); + } + } + + // Aggregate the snapshots + final List aggregatedSnapshotDtos = aggregate(snapshotsToAggregate); + + // get the details for this component from the last status history + final LinkedHashMap clusterStatusHistoryDetails = new LinkedHashMap<>(); + clusterStatusHistoryDetails.putAll(lastStatusHistory.getDetails()); + + final StatusHistoryDTO clusterStatusHistory = new StatusHistoryDTO(); + clusterStatusHistory.setGenerated(new Date()); + clusterStatusHistory.setFieldDescriptors(StatusHistoryUtil.createFieldDescriptorDtos(processorDescriptors)); + clusterStatusHistory.setDetails(clusterStatusHistoryDetails); + clusterStatusHistory.setStatusSnapshots(aggregatedSnapshotDtos); + + final ClusterStatusHistoryDTO history = new ClusterStatusHistoryDTO(); + history.setGenerated(new Date()); + history.setNodeStatusHistory(nodeHistories); + history.setClusterStatusHistory(clusterStatusHistory); + return history; + } + + public StatusHistoryDTO createStatusHistoryDto(final StatusHistory statusHistory) { + final StatusHistoryDTO dto = new StatusHistoryDTO(); + + dto.setDetails(new LinkedHashMap<>(statusHistory.getComponentDetails())); + dto.setFieldDescriptors(StatusHistoryUtil.createFieldDescriptorDtos(statusHistory)); + dto.setGenerated(statusHistory.getDateGenerated()); + + final List statusSnapshots = new ArrayList<>(); + for (final StatusSnapshot statusSnapshot : statusHistory.getStatusSnapshots()) { + statusSnapshots.add(StatusHistoryUtil.createStatusSnapshotDto(statusSnapshot)); + } + dto.setStatusSnapshots(statusSnapshots); + + return dto; + } + + public ClusterStatusHistoryDTO getConnectionStatusHistory(final String connectionId) { + return getConnectionStatusHistory(connectionId, null, null, Integer.MAX_VALUE); + } + + public ClusterStatusHistoryDTO getConnectionStatusHistory(final String connectionId, final Date startDate, final Date endDate, final int preferredDataPoints) { + final List nodeHistories = new ArrayList<>(); + + StatusHistoryDTO lastStatusHistory = null; + Set> connectionDescriptors = new LinkedHashSet<>(); + Map> snapshotsToAggregate = new TreeMap<>(); + + for (final Node node : getRawNodes()) { + final ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); + if (statusRepository == null) { + continue; + } + + final StatusHistory statusHistory = statusRepository.getConnectionStatusHistory(connectionId, startDate, endDate, preferredDataPoints); + if (statusHistory == null) { + continue; + } + + final StatusHistoryDTO statusHistoryDto = createStatusHistoryDto(statusHistory); + // record the status history (last) to get the componet details for use later + lastStatusHistory = statusHistoryDto; + connectionDescriptors.addAll(statusRepository.getConnectionMetricDescriptors()); + + final NodeStatusHistoryDTO nodeHistory = new NodeStatusHistoryDTO(); + nodeHistory.setStatusHistory(statusHistoryDto); + nodeHistory.setNode(createNodeDTO(node)); + nodeHistories.add(nodeHistory); + + // collect all of the snapshots to aggregate + for (final StatusSnapshot snapshot : statusHistory.getStatusSnapshots()) { + final Date normalizedDate = normalizeStatusSnapshotDate(snapshot.getTimestamp(), componentStatusSnapshotMillis); + List snapshots = snapshotsToAggregate.get(normalizedDate); + if (snapshots == null) { + snapshots = new ArrayList<>(); + snapshotsToAggregate.put(normalizedDate, snapshots); + } + snapshots.add(snapshot); + } + } + + // Aggregate the snapshots + final List aggregatedSnapshotDtos = aggregate(snapshotsToAggregate); + + // get the details for this component from the last status history + final LinkedHashMap clusterStatusHistoryDetails = new LinkedHashMap<>(); + clusterStatusHistoryDetails.putAll(lastStatusHistory.getDetails()); + + final StatusHistoryDTO clusterStatusHistory = new StatusHistoryDTO(); + clusterStatusHistory.setGenerated(new Date()); + clusterStatusHistory.setFieldDescriptors(StatusHistoryUtil.createFieldDescriptorDtos(connectionDescriptors)); + clusterStatusHistory.setDetails(clusterStatusHistoryDetails); + clusterStatusHistory.setStatusSnapshots(aggregatedSnapshotDtos); + + final ClusterStatusHistoryDTO history = new ClusterStatusHistoryDTO(); + history.setGenerated(new Date()); + history.setNodeStatusHistory(nodeHistories); + history.setClusterStatusHistory(clusterStatusHistory); + return history; + } + + public ClusterStatusHistoryDTO getProcessGroupStatusHistory(final String processGroupId) { + return getProcessGroupStatusHistory(processGroupId, null, null, Integer.MAX_VALUE); + } + + public ClusterStatusHistoryDTO getProcessGroupStatusHistory(final String processGroupId, final Date startDate, final Date endDate, final int preferredDataPoints) { + final List nodeHistories = new ArrayList<>(); + + StatusHistoryDTO lastStatusHistory = null; + Set> processGroupDescriptors = new LinkedHashSet<>(); + Map> snapshotsToAggregate = new TreeMap<>(); + + for (final Node node : getRawNodes()) { + final ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); + if (statusRepository == null) { + continue; + } + + final StatusHistory statusHistory = statusRepository.getProcessGroupStatusHistory(processGroupId, startDate, endDate, preferredDataPoints); + if (statusHistory == null) { + continue; + } + + final StatusHistoryDTO statusHistoryDto = createStatusHistoryDto(statusHistory); + // record the status history (last) to get the componet details for use later + lastStatusHistory = statusHistoryDto; + processGroupDescriptors.addAll(statusRepository.getProcessGroupMetricDescriptors()); + + final NodeStatusHistoryDTO nodeHistory = new NodeStatusHistoryDTO(); + nodeHistory.setStatusHistory(statusHistoryDto); + nodeHistory.setNode(createNodeDTO(node)); + nodeHistories.add(nodeHistory); + + // collect all of the snapshots to aggregate + for (final StatusSnapshot snapshot : statusHistory.getStatusSnapshots()) { + final Date normalizedDate = normalizeStatusSnapshotDate(snapshot.getTimestamp(), componentStatusSnapshotMillis); + List snapshots = snapshotsToAggregate.get(normalizedDate); + if (snapshots == null) { + snapshots = new ArrayList<>(); + snapshotsToAggregate.put(normalizedDate, snapshots); + } + snapshots.add(snapshot); + } + } + + // Aggregate the snapshots + final List aggregatedSnapshotDtos = aggregate(snapshotsToAggregate); + + // get the details for this component from the last status history + final LinkedHashMap clusterStatusHistoryDetails = new LinkedHashMap<>(); + clusterStatusHistoryDetails.putAll(lastStatusHistory.getDetails()); + + final StatusHistoryDTO clusterStatusHistory = new StatusHistoryDTO(); + clusterStatusHistory.setGenerated(new Date()); + clusterStatusHistory.setDetails(clusterStatusHistoryDetails); + clusterStatusHistory.setFieldDescriptors(StatusHistoryUtil.createFieldDescriptorDtos(processGroupDescriptors)); + clusterStatusHistory.setStatusSnapshots(aggregatedSnapshotDtos); + + final ClusterStatusHistoryDTO history = new ClusterStatusHistoryDTO(); + history.setGenerated(new Date()); + history.setNodeStatusHistory(nodeHistories); + history.setClusterStatusHistory(clusterStatusHistory); + return history; + } + + public ClusterStatusHistoryDTO getRemoteProcessGroupStatusHistory(final String remoteGroupId) { + return getRemoteProcessGroupStatusHistory(remoteGroupId, null, null, Integer.MAX_VALUE); + } + + public ClusterStatusHistoryDTO getRemoteProcessGroupStatusHistory(final String remoteGroupId, final Date startDate, final Date endDate, final int preferredDataPoints) { + final List nodeHistories = new ArrayList<>(); + + StatusHistoryDTO lastStatusHistory = null; + Set> remoteProcessGroupDescriptors = new LinkedHashSet<>(); + Map> snapshotsToAggregate = new TreeMap<>(); + + for (final Node node : getRawNodes()) { + final ComponentStatusRepository statusRepository = componentMetricsRepositoryMap.get(node.getNodeId()); + if (statusRepository == null) { + continue; + } + + final StatusHistory statusHistory = statusRepository.getRemoteProcessGroupStatusHistory(remoteGroupId, startDate, endDate, preferredDataPoints); + if (statusHistory == null) { + continue; + } + + final StatusHistoryDTO statusHistoryDto = createStatusHistoryDto(statusHistory); + // record the status history (last) to get the componet details for use later + lastStatusHistory = statusHistoryDto; + remoteProcessGroupDescriptors.addAll(statusRepository.getRemoteProcessGroupMetricDescriptors()); + + final NodeStatusHistoryDTO nodeHistory = new NodeStatusHistoryDTO(); + nodeHistory.setStatusHistory(statusHistoryDto); + nodeHistory.setNode(createNodeDTO(node)); + nodeHistories.add(nodeHistory); + + // collect all of the snapshots to aggregate + for (final StatusSnapshot snapshot : statusHistory.getStatusSnapshots()) { + final Date normalizedDate = normalizeStatusSnapshotDate(snapshot.getTimestamp(), componentStatusSnapshotMillis); + List snapshots = snapshotsToAggregate.get(normalizedDate); + if (snapshots == null) { + snapshots = new ArrayList<>(); + snapshotsToAggregate.put(normalizedDate, snapshots); + } + snapshots.add(snapshot); + } + } + + // Aggregate the snapshots + final List aggregatedSnapshotDtos = aggregate(snapshotsToAggregate); + + // get the details for this component from the last status history + final LinkedHashMap clusterStatusHistoryDetails = new LinkedHashMap<>(); + clusterStatusHistoryDetails.putAll(lastStatusHistory.getDetails()); + + final StatusHistoryDTO clusterStatusHistory = new StatusHistoryDTO(); + clusterStatusHistory.setGenerated(new Date()); + clusterStatusHistory.setDetails(clusterStatusHistoryDetails); + clusterStatusHistory.setFieldDescriptors(StatusHistoryUtil.createFieldDescriptorDtos(remoteProcessGroupDescriptors)); + clusterStatusHistory.setStatusSnapshots(aggregatedSnapshotDtos); + + final ClusterStatusHistoryDTO history = new ClusterStatusHistoryDTO(); + history.setGenerated(new Date()); + history.setNodeStatusHistory(nodeHistories); + history.setClusterStatusHistory(clusterStatusHistory); + return history; + } + + private static class ClusterManagerLock { + + private final Lock lock; + private static final Logger logger = LoggerFactory.getLogger("cluster.lock"); + private long lockTime; + private final String name; + + public ClusterManagerLock(final Lock lock, final String name) { + this.lock = lock; + this.name = name; + } + + @SuppressWarnings("unused") + public boolean tryLock() { + logger.trace("Trying to obtain Cluster Manager Lock: {}", name); + final boolean success = lock.tryLock(); + if (!success) { + logger.trace("TryLock failed for Cluster Manager Lock: {}", name); + return false; + } + logger.trace("TryLock successful"); + return true; + } + + public boolean tryLock(final long timeout, final TimeUnit timeUnit) { + logger.trace("Trying to obtain Cluster Manager Lock {} with a timeout of {} {}", name, timeout, timeUnit); + final boolean success; + try { + success = lock.tryLock(timeout, timeUnit); + } catch (final InterruptedException ie) { + return false; + } + + if (!success) { + logger.trace("TryLock failed for Cluster Manager Lock {} with a timeout of {} {}", name, timeout, timeUnit); + return false; + } + logger.trace("TryLock successful"); + return true; + } + + public void lock() { + logger.trace("Obtaining Cluster Manager Lock {}", name); + lock.lock(); + lockTime = System.nanoTime(); + logger.trace("Obtained Cluster Manager Lock {}", name); + } + + public void unlock(final String task) { + logger.trace("Releasing Cluster Manager Lock {}", name); + final long nanosLocked = System.nanoTime() - lockTime; + lock.unlock(); + logger.trace("Released Cluster Manager Lock {}", name); + + final long millisLocked = TimeUnit.MILLISECONDS.convert(nanosLocked, TimeUnit.NANOSECONDS); + if (millisLocked > 100L) { + logger.debug("Cluster Manager Lock {} held for {} milliseconds for task: {}", name, millisLocked, task); + } + } + } + + @Override + public Set getControllerServiceIdentifiers(final Class serviceType) { + return controllerServiceProvider.getControllerServiceIdentifiers(serviceType); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java new file mode 100644 index 0000000000..84565da137 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/node/Node.java @@ -0,0 +1,252 @@ +/* + * 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.cluster.node; + +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.nifi.cluster.HeartbeatPayload; +import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Represents a connected flow controller. Nodes always have an immutable + * identifier and a status. The status may be changed, but never null. + * + * A Node may be cloned, but the cloning is a shallow copy of the instance. + * + * This class overrides hashCode and equals and considers two instances to be + * equal if they have the equal NodeIdentifiers. + * + * @author unattributed + */ +public class Node implements Cloneable, Comparable { + + private static final Logger lockLogger = LoggerFactory.getLogger("cluster.lock"); + + /** + * The semantics of a Node status are as follows: + *
    + *
  • CONNECTED -- a flow controller that is connected to the cluster. A + * connecting node transitions to connected after the cluster receives the + * flow controller's first heartbeat. A connected node can transition to + * disconnecting.
  • + *
  • CONNECTING -- a flow controller has issued a connection request to + * the cluster, but has not yet sent a heartbeat. A connecting node can + * transition to disconnecting or connected. The cluster will not accept any + * external requests to change the flow while any node is connecting.
  • + *
  • DISCONNECTED -- a flow controller that is not connected to the + * cluster. A disconnected node can transition to connecting.
  • + *
  • DISCONNECTING -- a flow controller that is in the process of + * disconnecting from the cluster. A disconnecting node will always + * transition to disconnected.
  • + *
+ */ + public static enum Status { + + CONNECTED, + CONNECTING, + DISCONNECTED, + DISCONNECTING + } + + /** + * the node's unique identifier + */ + private final NodeIdentifier nodeId; + + /** + * the node statue + */ + private Status status; + + /** + * the last heartbeat received by from the node + */ + private Heartbeat lastHeartbeat; + + /** + * the payload of the last heartbeat received from the node + */ + private HeartbeatPayload lastHeartbeatPayload; + + /** + * the last time the connection for this node was requested + */ + private AtomicLong connectionRequestedTimestamp = new AtomicLong(0L); + + /** + * a flag to indicate this node was disconnected because of a lack of + * heartbeat + */ + private boolean heartbeatDisconnection; + + public Node(final NodeIdentifier id, final Status status) { + if (id == null) { + throw new IllegalArgumentException("ID may not be null."); + } else if (status == null) { + throw new IllegalArgumentException("Status may not be null."); + } + this.nodeId = id; + this.status = status; + } + + public NodeIdentifier getNodeId() { + return nodeId; + } + + /** + * Returns the last received heartbeat or null if no heartbeat has been set. + * + * @return a heartbeat or null + */ + public Heartbeat getHeartbeat() { + return lastHeartbeat; + } + + public HeartbeatPayload getHeartbeatPayload() { + return lastHeartbeatPayload; + } + + /** + * Sets the last heartbeat received. + * + * @param heartbeat a heartbeat + * + * @throws ProtocolException if the heartbeat's payload failed unmarshalling + */ + public void setHeartbeat(final Heartbeat heartbeat) throws ProtocolException { + this.lastHeartbeat = heartbeat; + if (this.lastHeartbeat == null) { + this.lastHeartbeatPayload = null; + } else { + final byte[] payload = lastHeartbeat.getPayload(); + if (payload == null || payload.length == 0) { + this.lastHeartbeatPayload = null; + } else { + this.lastHeartbeatPayload = HeartbeatPayload.unmarshal(payload); + } + } + } + + /** + * Returns the time of the last received connection request for this node. + * + * @return the time when the connection request for this node was received. + */ + public long getConnectionRequestedTimestamp() { + return connectionRequestedTimestamp.get(); + } + + /** + * Sets the time when the connection request for this node was last + * received. + * + * This method is thread-safe and may be called without obtaining any lock. + * + * @param connectionRequestedTimestamp + */ + public void setConnectionRequestedTimestamp(long connectionRequestedTimestamp) { + this.connectionRequestedTimestamp.set(connectionRequestedTimestamp); + } + + /** + * Returns true if the node was disconnected due to lack of heartbeat; false + * otherwise. + * + * @return true if the node was disconnected due to lack of heartbeat; false + * otherwise. + */ + public boolean isHeartbeatDisconnection() { + return heartbeatDisconnection; + } + + /** + * Sets the status to disconnected and flags the node as being disconnected + * by lack of heartbeat. + */ + public void setHeartbeatDisconnection() { + setStatus(Status.DISCONNECTED); + heartbeatDisconnection = true; + } + + /** + * @return the status + */ + public Status getStatus() { + return status; + } + + /** + * @param status a status + */ + public void setStatus(final Status status) { + if (status == null) { + throw new IllegalArgumentException("Status may not be null."); + } + this.status = status; + heartbeatDisconnection = false; + } + + @Override + public Node clone() { + final Node clone = new Node(nodeId, status); + clone.lastHeartbeat = lastHeartbeat; + clone.lastHeartbeatPayload = lastHeartbeatPayload; + clone.heartbeatDisconnection = heartbeatDisconnection; + clone.connectionRequestedTimestamp = connectionRequestedTimestamp; + return clone; + } + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final Node other = (Node) obj; + if (this.nodeId != other.nodeId && (this.nodeId == null || !this.nodeId.equals(other.nodeId))) { + return false; + } + return true; + } + + @Override + public int hashCode() { + int hash = 7; + hash = 53 * hash + (this.nodeId != null ? this.nodeId.hashCode() : 0); + return hash; + } + + @Override + public String toString() { + return nodeId.toString(); + } + + @Override + public int compareTo(final Node o) { + if (o == null) { + return -1; + } + return getNodeId().getId().compareTo(o.getNodeId().getId()); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java new file mode 100644 index 0000000000..e26d196561 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/ClusterManagerProtocolServiceLocatorFactoryBean.java @@ -0,0 +1,116 @@ +/* + * 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.cluster.spring; + +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.cluster.protocol.impl.ClusterServiceDiscovery; +import org.apache.nifi.cluster.protocol.impl.ClusterServiceLocator; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.NiFiProperties; + +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.DisposableBean; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * Factory bean for creating a singleton ClusterManagerProtocolServiceLocator + * instance. If the application is configured to act as the cluster manager, + * then null is always returned as the created instance. + * + * The cluster manager protocol service represents the socket endpoint for + * sending internal socket messages to the cluster manager. + */ +public class ClusterManagerProtocolServiceLocatorFactoryBean implements FactoryBean, ApplicationContextAware, DisposableBean { + + private ApplicationContext applicationContext; + + private ClusterServiceLocator locator; + + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + /* + * If configured for the cluster manager, then the service locator is never used. + */ + if (properties.isClusterManager()) { + return null; + } else if (locator == null) { + + if (properties.getClusterProtocolUseMulticast()) { + + // get the service discovery instance + final ClusterServiceDiscovery serviceDiscovery = applicationContext.getBean("clusterManagerProtocolServiceDiscovery", ClusterServiceDiscovery.class); + + // create service location configuration + final ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); + config.setNumAttempts(properties.getClusterProtocolMulticastServiceLocatorAttempts()); + + final int delay = (int) FormatUtils.getTimeDuration(properties.getClusterProtocolMulticastServiceLocatorAttemptsDelay(), TimeUnit.SECONDS); + config.setTimeBetweenAttempts(delay); + config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); + + locator = new ClusterServiceLocator(serviceDiscovery); + locator.setAttemptsConfig(config); + + } else { + final String serviceName = applicationContext.getBean("clusterManagerProtocolServiceName", String.class); + final InetSocketAddress serviceAddress = properties.getClusterNodeUnicastManagerProtocolAddress(); + final DiscoverableService service = new DiscoverableServiceImpl(serviceName, serviceAddress); + locator = new ClusterServiceLocator(service); + } + + // start the locator + locator.start(); + + } + return locator; + + } + + @Override + public Class getObjectType() { + return ClusterServiceLocator.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + @Override + public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { + this.applicationContext = applicationContext; + } + + @Override + public void destroy() throws Exception { + if (locator != null && locator.isRunning()) { + locator.stop(); + } + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java new file mode 100644 index 0000000000..ef722986fb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/FileBasedClusterNodeFirewallFactoryBean.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.spring; + +import java.io.File; +import org.apache.nifi.cluster.firewall.impl.FileBasedClusterNodeFirewall; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.factory.FactoryBean; + +/** + * Factory bean for creating a singleton FileBasedClusterNodeFirewall instance. + */ +public class FileBasedClusterNodeFirewallFactoryBean implements FactoryBean { + + private FileBasedClusterNodeFirewall firewall; + + private NiFiProperties properties; + + @Override + public Object getObject() throws Exception { + if (firewall == null) { + final File config = properties.getClusterManagerNodeFirewallFile(); + final File restoreDirectory = properties.getRestoreDirectory(); + if (config != null) { + firewall = new FileBasedClusterNodeFirewall(config, restoreDirectory); + } + } + return firewall; + } + + @Override + public Class getObjectType() { + return FileBasedClusterNodeFirewall.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java new file mode 100644 index 0000000000..7169730a56 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/cluster/spring/WebClusterManagerFactoryBean.java @@ -0,0 +1,139 @@ +/* + * 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.cluster.spring; + +import java.nio.file.Paths; +import org.apache.nifi.admin.service.AuditService; +import org.apache.nifi.cluster.event.EventManager; +import org.apache.nifi.cluster.firewall.ClusterNodeFirewall; +import org.apache.nifi.cluster.flow.DataFlowManagementService; +import org.apache.nifi.cluster.manager.HttpRequestReplicator; +import org.apache.nifi.cluster.manager.HttpResponseMapper; +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderListener; +import org.apache.nifi.cluster.protocol.impl.ClusterServicesBroadcaster; +import org.apache.nifi.controller.service.ControllerServiceLoader; +import org.apache.nifi.encrypt.StringEncryptor; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.util.NiFiProperties; +import org.springframework.beans.BeansException; +import org.springframework.beans.factory.FactoryBean; +import org.springframework.context.ApplicationContext; +import org.springframework.context.ApplicationContextAware; + +/** + * Factory bean for creating a singleton WebClusterManager instance. If the + * application is not configured to act as the cluster manager, then null is + * always returned as the created instance. + */ +public class WebClusterManagerFactoryBean implements FactoryBean, ApplicationContextAware { + + private ApplicationContext applicationContext; + + private WebClusterManager clusterManager; + + private NiFiProperties properties; + + private StringEncryptor encryptor; + + @Override + public Object getObject() throws Exception { + if (properties.isClusterManager() && properties.isNode()) { + throw new IllegalStateException("Application may be configured as a cluster manager or a node, but not both."); + } else if (!properties.isClusterManager()) { + /* + * If not configured for the cluster manager, then the cluster manager is never used. + * null is returned so that we don't instantiate a thread pool or other resources. + */ + return null; + } else if (clusterManager == null) { + + // get the service configuration path (fail early) + final String serviceConfigurationFile = properties.getProperty(NiFiProperties.SERVICE_CONFIGURATION_FILE); + if (serviceConfigurationFile == null) { + throw new NullPointerException("The service configuration file has not been specified."); + } + + final HttpRequestReplicator requestReplicator = applicationContext.getBean("httpRequestReplicator", HttpRequestReplicator.class); + final HttpResponseMapper responseMapper = applicationContext.getBean("httpResponseMapper", HttpResponseMapper.class); + final DataFlowManagementService dataFlowService = applicationContext.getBean("dataFlowManagementService", DataFlowManagementService.class); + final ClusterManagerProtocolSenderListener senderListener = applicationContext.getBean("clusterManagerProtocolSenderListener", ClusterManagerProtocolSenderListener.class); + + // create the manager + clusterManager = new WebClusterManager( + requestReplicator, + responseMapper, + dataFlowService, + senderListener, + properties, + encryptor + ); + + // set the service broadcaster + if (properties.getClusterProtocolUseMulticast()) { + + // create broadcaster + final ClusterServicesBroadcaster broadcaster = applicationContext.getBean("clusterServicesBroadcaster", ClusterServicesBroadcaster.class); + + // register the cluster manager protocol service + final String clusterManagerProtocolServiceName = applicationContext.getBean("clusterManagerProtocolServiceName", String.class); + final DiscoverableService clusterManagerProtocolService = new DiscoverableServiceImpl(clusterManagerProtocolServiceName, properties.getClusterManagerProtocolAddress()); + broadcaster.addService(clusterManagerProtocolService); + + clusterManager.setServicesBroadcaster(broadcaster); + } + + // set the event manager + clusterManager.setEventManager(applicationContext.getBean("nodeEventHistoryManager", EventManager.class)); + + // set the cluster firewall + clusterManager.setClusterFirewall(applicationContext.getBean("clusterFirewall", ClusterNodeFirewall.class)); + + // set the audit service + clusterManager.setAuditService(applicationContext.getBean("auditService", AuditService.class)); + + // load the controller services + final ControllerServiceLoader serviceLoader = new ControllerServiceLoader(Paths.get(serviceConfigurationFile)); + serviceLoader.loadControllerServices(clusterManager); + } + return clusterManager; + } + + @Override + public Class getObjectType() { + return WebClusterManager.class; + } + + @Override + public boolean isSingleton() { + return true; + } + + @Override + public void setApplicationContext(ApplicationContext applicationContext) throws BeansException { + this.applicationContext = applicationContext; + } + + public void setProperties(NiFiProperties properties) { + this.properties = properties; + } + + public void setEncryptor(final StringEncryptor encryptor) { + this.encryptor = encryptor; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java new file mode 100644 index 0000000000..1ed5b30054 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/java/org/apache/nifi/controller/reporting/ClusteredReportingTaskNode.java @@ -0,0 +1,49 @@ +/* + * 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.controller.reporting; + +import org.apache.nifi.cluster.manager.impl.ClusteredReportingContext; +import org.apache.nifi.controller.ProcessScheduler; +import org.apache.nifi.controller.ValidationContextFactory; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.reporting.BulletinRepository; +import org.apache.nifi.reporting.EventAccess; +import org.apache.nifi.reporting.ReportingContext; +import org.apache.nifi.reporting.ReportingTask; + +public class ClusteredReportingTaskNode extends AbstractReportingTaskNode { + + private final EventAccess eventAccess; + private final BulletinRepository bulletinRepository; + private final ControllerServiceProvider serviceProvider; + + public ClusteredReportingTaskNode(final ReportingTask reportingTask, final String id, final ProcessScheduler scheduler, + final EventAccess eventAccess, final BulletinRepository bulletinRepository, final ControllerServiceProvider serviceProvider, + final ValidationContextFactory validationContextFactory) { + super(reportingTask, id, serviceProvider, scheduler, validationContextFactory); + + this.eventAccess = eventAccess; + this.bulletinRepository = bulletinRepository; + this.serviceProvider = serviceProvider; + } + + @Override + public ReportingContext getReportingContext() { + return new ClusteredReportingContext(eventAccess, bulletinRepository, getProperties(), serviceProvider); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml b/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml new file mode 100644 index 0000000000..68c29bca00 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/main/resources/nifi-cluster-manager-context.xml @@ -0,0 +1,124 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java new file mode 100644 index 0000000000..09ea44b383 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/event/impl/EventManagerImplTest.java @@ -0,0 +1,119 @@ +/* + * 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.cluster.event.impl; + +import org.apache.nifi.cluster.event.impl.EventManagerImpl; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.nifi.cluster.event.Event; +import org.apache.nifi.cluster.event.Event.Category; +import org.apache.nifi.cluster.event.EventManager; +import org.junit.Test; +import static org.junit.Assert.*; + +/** + * @author unattributed + */ +public class EventManagerImplTest { + + @Test(expected = IllegalArgumentException.class) + public void testNonPositiveHistorySize() { + new EventManagerImpl(0); + } + + @Test + public void testGetEventsUnknownSource() { + EventManager manager = new EventManagerImpl(1); + assertEquals(Collections.EMPTY_LIST, manager.getEvents("unknown value")); + } + + @Test + public void testGetEvents() { + + EventManager manager = new EventManagerImpl(2); + + Event e1 = new Event("1", "Event1", Category.INFO, 0); + Event e2 = new Event("1", "Event2", Category.INFO, 1); + + manager.addEvent(e1); + manager.addEvent(e2); + + List events = manager.getEvents("1"); + + // assert newest to oldest + assertEquals(Arrays.asList(e2, e1), events); + } + + @Test + public void testGetMostRecentEventUnknownSource() { + EventManager manager = new EventManagerImpl(1); + assertNull(manager.getMostRecentEvent("unknown value")); + } + + @Test + public void testGetMostRecentEvent() { + + EventManager manager = new EventManagerImpl(2); + + Event e1 = new Event("1", "Event1", Category.INFO, 0); + Event e2 = new Event("1", "Event2", Category.INFO, 1); + + manager.addEvent(e1); + manager.addEvent(e2); + + // assert newest to oldest + assertEquals(e2, manager.getMostRecentEvent("1")); + } + + @Test + public void testAddEventExceedsHistorySize() { + + EventManager manager = new EventManagerImpl(1); + + Event e1 = new Event("1", "Event1", Category.INFO, 0); + Event e2 = new Event("1", "Event2", Category.INFO, 1); + + manager.addEvent(e1); + manager.addEvent(e2); + + List events = manager.getEvents("1"); + + // assert oldest evicted + assertEquals(Arrays.asList(e2), events); + + } + + @Test + public void testClearHistory() { + + EventManager manager = new EventManagerImpl(1); + + Event e1 = new Event("1", "Event1", Category.INFO, 0); + Event e2 = new Event("1", "Event2", Category.INFO, 1); + + manager.addEvent(e1); + manager.addEvent(e2); + + manager.clearEventHistory("1"); + + // assert oldest evicted + assertTrue(manager.getEvents("1").isEmpty()); + + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java new file mode 100644 index 0000000000..2fcf7ef184 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/firewall/impl/FileBasedClusterNodeFirewallTest.java @@ -0,0 +1,99 @@ +/* + * 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.cluster.firewall.impl; + +import org.apache.nifi.cluster.firewall.impl.FileBasedClusterNodeFirewall; +import java.io.File; +import java.io.IOException; +import org.apache.nifi.file.FileUtils; +import org.junit.After; +import org.junit.Before; +import static org.junit.Assert.*; +import org.junit.Test; + +public class FileBasedClusterNodeFirewallTest { + + private FileBasedClusterNodeFirewall ipsFirewall; + + private FileBasedClusterNodeFirewall acceptAllFirewall; + + private File ipsConfig; + + private File emptyConfig; + + private File restoreDirectory; + + @Before + public void setup() throws Exception { + + ipsConfig = new File("src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt"); + emptyConfig = new File("src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt"); + + restoreDirectory = new File(System.getProperty("java.io.tmpdir") + "/firewall_restore"); + + ipsFirewall = new FileBasedClusterNodeFirewall(ipsConfig, restoreDirectory); + acceptAllFirewall = new FileBasedClusterNodeFirewall(emptyConfig); + } + + @After + public void teardown() throws IOException { + deleteFile(restoreDirectory); + } + + @Test + public void testSyncWithRestore() { + assertEquals(ipsConfig.length(), new File(restoreDirectory, ipsConfig.getName()).length()); + } + + @Test + public void testIsPermissibleWithExactMatch() { + assertTrue(ipsFirewall.isPermissible("2.2.2.2")); + } + + @Test + public void testIsPermissibleWithSubnetMatch() { + assertTrue(ipsFirewall.isPermissible("3.3.3.255")); + } + + @Test + public void testIsPermissibleWithNoMatch() { + assertFalse(ipsFirewall.isPermissible("255.255.255.255")); + } + + @Test + public void testIsPermissibleWithMalformedData() { + assertFalse(ipsFirewall.isPermissible("abc")); + } + + @Test + public void testIsPermissibleWithEmptyConfig() { + assertTrue(acceptAllFirewall.isPermissible("1.1.1.1")); + } + + @Test + public void testIsPermissibleWithEmptyConfigWithMalformedData() { + assertTrue(acceptAllFirewall.isPermissible("abc")); + } + + private boolean deleteFile(final File file) { + if (file.isDirectory()) { + FileUtils.deleteFilesInDir(file, null, null, true, true); + } + return FileUtils.deleteFile(file, null, 10); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java new file mode 100644 index 0000000000..6294dfc0da --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/flow/impl/DataFlowManagementServiceImplTest.java @@ -0,0 +1,341 @@ +/* + * 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.cluster.flow.impl; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import javax.xml.parsers.ParserConfigurationException; + +import org.apache.nifi.cluster.flow.DataFlowDao; +import org.apache.nifi.cluster.flow.PersistedFlowState; +import org.apache.nifi.cluster.protocol.ClusterManagerProtocolSender; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.impl.ClusterManagerProtocolSenderImpl; +import org.apache.nifi.cluster.protocol.impl.SocketProtocolListener; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketConfiguration; + +import org.apache.commons.io.FileUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.xml.sax.SAXException; + +/** + * @author unattributed + */ +public class DataFlowManagementServiceImplTest { + + private DataFlowManagementServiceImpl service; + private File restoreLocation; + private File primaryLocation; + private DataFlowDao dao; + private int apiDummyPort; + private int socketPort; + private SocketConfiguration socketConfig; + private ClusterManagerProtocolSender sender; + private ServerSocketConfiguration serverSocketConfig; + private SocketProtocolListener listener; + + @Before + public void setup() throws IOException { + + primaryLocation = new File(System.getProperty("java.io.tmpdir") + "/primary"); + restoreLocation = new File(System.getProperty("java.io.tmpdir") + "/restore"); + + FileUtils.deleteDirectory(primaryLocation); + FileUtils.deleteDirectory(restoreLocation); + + ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + socketConfig = new SocketConfiguration(); + socketConfig.setSocketTimeout(1000); + serverSocketConfig = new ServerSocketConfiguration(); + + dao = new DataFlowDaoImpl(primaryLocation, restoreLocation, false); + + sender = new ClusterManagerProtocolSenderImpl(socketConfig, protocolContext); + + service = new DataFlowManagementServiceImpl(dao, sender); + service.start(); + + listener = new SocketProtocolListener(1, 0, serverSocketConfig, protocolContext); + listener.start(); + + apiDummyPort = 7777; + socketPort = listener.getPort(); + } + + @After + public void teardown() throws IOException { + + if (service != null && service.isRunning()) { + service.stop(); + } + + if (listener != null && listener.isRunning()) { + try { + listener.stop(); + } catch (final Exception ex) { + ex.printStackTrace(System.out); + } + } + + } + + @Test + public void testLoadFlowWithNonExistentFlow() throws ParserConfigurationException, SAXException, IOException { + verifyFlow(); + } + + @Test + public void testLoadFlowWithNonExistentFlowWhenServiceStopped() throws IOException, SAXException, ParserConfigurationException { + service.stop(); + verifyFlow(); + } + + private void verifyFlow() throws ParserConfigurationException, SAXException, IOException { + final byte[] flowBytes = service.loadDataFlow().getDataFlow().getFlow(); + final DocumentBuilder docBuilder = DocumentBuilderFactory.newInstance().newDocumentBuilder(); + final Document doc = docBuilder.parse(new ByteArrayInputStream(flowBytes)); + final Element controller = (Element) doc.getElementsByTagName("flowController").item(0); + final Element rootGroup = (Element) controller.getElementsByTagName("rootGroup").item(0); + final String rootGroupName = rootGroup.getElementsByTagName("name").item(0).getTextContent(); + assertEquals("NiFi Flow", rootGroupName); + } + + @Test + public void testLoadFlowSingleNode() throws Exception { + String flowStr = ""; + byte[] flowBytes = flowStr.getBytes(); + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + + NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort); + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId))); + service.setPersistedFlowState(PersistedFlowState.STALE); + + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.CURRENT); + + assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); + assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); + + } + + @Test + public void testLoadFlowWithSameNodeIds() throws Exception { + + String flowStr = ""; + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + + NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort); + NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort); + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); + service.setPersistedFlowState(PersistedFlowState.STALE); + + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.CURRENT); + + // verify that flow is current + assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); + + // add same ids in different order + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId2, nodeId1))); + + // verify flow is still current + assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); + + } + + @Test + public void testLoadFlowWithABadNode() throws Exception { + + String flowStr = ""; + byte[] flowBytes = flowStr.getBytes(); + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + + NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1); + NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort); + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); + service.setPersistedFlowState(PersistedFlowState.STALE); + + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.CURRENT); + + assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); + assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); + + } + + @Test + public void testLoadFlowWithConstantNodeIdChanging() throws Exception { + String flowStr = ""; + byte[] flowBytes = flowStr.getBytes(); + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + + NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1); + NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort); + + for (int i = 0; i < 1000; i++) { + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); + service.setPersistedFlowState(PersistedFlowState.STALE); + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + } + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.CURRENT); + + assertEquals(PersistedFlowState.CURRENT, service.getPersistedFlowState()); + assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); + } + + @Test + public void testLoadFlowWithConstantNodeIdChangingWithRetrievalDelay() throws Exception { + + String flowStr = ""; + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + + NodeIdentifier nodeId1 = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1); + NodeIdentifier nodeId2 = new NodeIdentifier("2", "localhost", apiDummyPort, "localhost", socketPort); + + service.setRetrievalDelay("5 sec"); + for (int i = 0; i < 1000; i++) { + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId1, nodeId2))); + service.setPersistedFlowState(PersistedFlowState.STALE); + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + } + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.STALE); + + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + } + + @Test + public void testStopRequestedWhileRetrieving() throws Exception { + + String flowStr = ""; + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + Set nodeIds = new HashSet<>(); + for (int i = 0; i < 1000; i++) { + nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort + 1)); + } + nodeIds.add(new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort)); + + long lastRetrievalTime = service.getLastRetrievalTime(); + + service.setNodeIds(nodeIds); + service.setPersistedFlowState(PersistedFlowState.STALE); + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.STALE); + + service.stop(); + + service.setPersistedFlowState(PersistedFlowState.STALE); + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + assertEquals(lastRetrievalTime, service.getLastRetrievalTime()); + + } + + @Test + public void testLoadFlowUnknownState() throws Exception { + + String flowStr = ""; + byte[] flowBytes = flowStr.getBytes(); + listener.addHandler(new FlowRequestProtocolHandler(new StandardDataFlow(flowStr.getBytes("UTF-8"), new byte[0], new byte[0]))); + NodeIdentifier nodeId = new NodeIdentifier("1", "localhost", apiDummyPort, "localhost", socketPort); + + service.setNodeIds(new HashSet<>(Arrays.asList(nodeId))); + service.setPersistedFlowState(PersistedFlowState.STALE); + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + service.setPersistedFlowState(PersistedFlowState.UNKNOWN); + + assertEquals(PersistedFlowState.UNKNOWN, service.getPersistedFlowState()); + + service.setPersistedFlowState(PersistedFlowState.STALE); + assertEquals(PersistedFlowState.STALE, service.getPersistedFlowState()); + + // sleep long enough for the flow retriever to run + waitForState(PersistedFlowState.CURRENT); + + assertArrayEquals(flowBytes, service.loadDataFlow().getDataFlow().getFlow()); + + } + + private class FlowRequestProtocolHandler implements ProtocolHandler { + + private StandardDataFlow dataFlow; + + public FlowRequestProtocolHandler(final StandardDataFlow dataFlow) { + this.dataFlow = dataFlow; + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + FlowResponseMessage response = new FlowResponseMessage(); + response.setDataFlow(dataFlow); + return response; + } + + } + + private void waitForState(PersistedFlowState state) throws InterruptedException { + for (int i = 0; i < 30; i++) { + if (service.getPersistedFlowState() == state) { + break; + } else { + Thread.sleep(1000); + } + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java new file mode 100644 index 0000000000..0c65aba218 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpRequestReplicatorImplTest.java @@ -0,0 +1,368 @@ +/* + * 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.cluster.manager.impl; + +import org.apache.nifi.cluster.manager.impl.HttpRequestReplicatorImpl; +import javax.ws.rs.core.Response; +import javax.xml.bind.annotation.XmlRootElement; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.MultivaluedMap; +import java.net.URISyntaxException; +import java.util.Arrays; +import java.util.Iterator; +import javax.ws.rs.core.StreamingOutput; +import org.apache.nifi.cluster.manager.testutils.HttpResponse; +import org.apache.nifi.cluster.manager.testutils.HttpServer; +import com.sun.jersey.api.client.Client; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.ws.rs.core.Response.Status; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.apache.nifi.cluster.manager.testutils.HttpResponseAction; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import static org.junit.Assert.*; + +/** + * @author unattributed + */ +public class HttpRequestReplicatorImplTest { + + private Client client; + private HttpRequestReplicatorImpl replicator; + private int executorThreadCount; + private int serverThreadCount; + private int serverPort; + private HttpServer server; + private Map> expectedRequestParameters; + private Map expectedRequestHeaders; + private Map expectedResponseHeaders; + private Object expectedEntity; + private String expectedBody; + private URI prototypeUri; + + @Before + public void setUp() throws IOException, URISyntaxException { + + executorThreadCount = 5; + serverThreadCount = 3; + + client = Client.create(); + + replicator = new HttpRequestReplicatorImpl(executorThreadCount, client, "1 sec", "1 sec"); + replicator.start(); + + expectedRequestHeaders = new HashMap<>(); + expectedRequestHeaders.put("header1", "header value1"); + expectedRequestHeaders.put("header2", "header value2"); + + expectedRequestParameters = new HashMap<>(); + expectedRequestParameters.put("param1", Arrays.asList("p value1")); + expectedRequestParameters.put("param2", Arrays.asList("p value2")); + + expectedResponseHeaders = new HashMap<>(); + expectedResponseHeaders.put("header1", "header value1"); + expectedResponseHeaders.put("header2", "header value2"); + + expectedEntity = new Entity(); + + expectedBody = "some text"; + + prototypeUri = new URI("http://prototype.host/path/to/resource"); + + server = new HttpServer(serverThreadCount, 0); + server.start(); + serverPort = server.getPort(); + } + + @After + public void teardown() { + if (server.isRunning()) { + server.stop(); + } + if (replicator.isRunning()) { + replicator.stop(); + } + } + + @Test + public void testReplicateGetLessNodesThanReplicatorThreads() throws Throwable { + testReplicateXXX(executorThreadCount - 1, HttpMethod.GET); + } + + @Test + public void testReplicateGetMoreNodesThanReplicatorThreads() throws Throwable { + testReplicateXXX(executorThreadCount + 1, HttpMethod.GET); + } + + @Test + public void testReplicateGetWithUnresponsiveNode() throws Throwable { + + // nodes + Set nodeIds = createNodes(2, "localhost", serverPort); + + // response + HttpResponse expectedResponse = new HttpResponse(Status.OK, expectedBody); + + // first response normal, second response slow + server.addResponseAction(new HttpResponseAction(expectedResponse)); + server.addResponseAction(new HttpResponseAction(expectedResponse, 3500)); + + Set responses = replicator.replicate( + nodeIds, + HttpMethod.GET, + prototypeUri, + expectedRequestParameters, + expectedRequestHeaders); + + assertEquals(nodeIds.size(), responses.size()); + + Iterator nodeResponseItr = responses.iterator(); + + NodeResponse firstResponse = nodeResponseItr.next(); + NodeResponse secondResponse = nodeResponseItr.next(); + NodeResponse goodResponse; + NodeResponse badResponse; + if (firstResponse.hasThrowable()) { + goodResponse = secondResponse; + badResponse = firstResponse; + } else { + goodResponse = firstResponse; + badResponse = secondResponse; + } + + // good response + // check status + assertEquals(Status.OK.getStatusCode(), goodResponse.getStatus()); + + // check entity stream + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + ((StreamingOutput) goodResponse.getResponse().getEntity()).write(baos); + assertEquals("some text", new String(baos.toByteArray())); + + // bad response + assertTrue(badResponse.hasThrowable()); + assertEquals(Status.INTERNAL_SERVER_ERROR.getStatusCode(), badResponse.getStatus()); + + } + + @Test(expected = IllegalArgumentException.class) + public void testReplicateGetWithEntity() throws Throwable { + testReplicateXXXEntity(HttpMethod.GET); + } + + @Test + public void testReplicatePost() throws Throwable { + testReplicateXXX(HttpMethod.POST); + } + + @Test + public void testReplicatePostWithEntity() throws Throwable { + testReplicateXXXEntity(HttpMethod.POST); + } + + @Test + public void testReplicatePut() throws Throwable { + testReplicateXXX(HttpMethod.PUT); + } + + @Test + public void testReplicatePutWithEntity() throws Throwable { + testReplicateXXXEntity(HttpMethod.PUT); + } + + @Test + public void testReplicateDelete() throws Throwable { + testReplicateXXX(HttpMethod.DELETE); + } + + @Test(expected = IllegalArgumentException.class) + public void testReplicateDeleteWithEntity() throws Throwable { + testReplicateXXXEntity(HttpMethod.DELETE); + } + + @Test + public void testReplicateHead() throws Throwable { + testReplicateXXX(HttpMethod.HEAD); + } + + @Test(expected = IllegalArgumentException.class) + public void testReplicateHeadWithEntity() throws Throwable { + testReplicateXXXEntity(HttpMethod.HEAD); + } + + @Test + public void testReplicateOptions() throws Throwable { + testReplicateXXX(HttpMethod.OPTIONS); + } + + @Test(expected = IllegalArgumentException.class) + public void testReplicateOptionsWithEntity() throws Throwable { + testReplicateXXXEntity(HttpMethod.OPTIONS); + } + + private void testReplicateXXX(final String method) throws Throwable { + testReplicateXXX(executorThreadCount, method); + } + + private void testReplicateXXX(final int numNodes, final String method) throws Throwable { + + // nodes + Set nodeIds = createNodes(numNodes, "localhost", serverPort); + + // set up responses + for (int i = 0; i < nodeIds.size(); i++) { + HttpResponse response = new HttpResponse(Status.OK, expectedBody); + response.addHeaders(expectedResponseHeaders); + server.addResponseAction(new HttpResponseAction(response)); + } + + // setup request parameters + server.addCheckedParameters(expectedRequestParameters); + + // request headers + server.addCheckedHeaders(expectedRequestHeaders); + + Set responses = replicator.replicate( + nodeIds, + method, + prototypeUri, + expectedRequestParameters, + expectedRequestHeaders); + + Set returnedNodeIds = new HashSet<>(); + for (NodeResponse response : responses) { + + // check if we received an exception + if (response.hasThrowable()) { + throw response.getThrowable(); + } + + // gather ids to verify later + returnedNodeIds.add(response.getNodeId()); + + // check status + assertEquals(Status.OK.getStatusCode(), response.getStatus()); + + Response serverResponse = response.getResponse(); + + // check response headers are copied + assertTrue(containsHeaders(expectedResponseHeaders, serverResponse.getMetadata())); + + // check entity stream + if (HttpMethod.HEAD.equalsIgnoreCase(method)) { + assertNull(serverResponse.getEntity()); + } else { + assertTrue(isEquals((StreamingOutput) serverResponse.getEntity(), expectedBody)); + } + + } + + // check node Ids + assertEquals(nodeIds, returnedNodeIds); + } + + private void testReplicateXXXEntity(final String method) throws Throwable { + testReplicateXXXEntity(executorThreadCount, method); + } + + private void testReplicateXXXEntity(final int numNodes, final String method) throws Throwable { + + // nodes + Set nodeIds = createNodes(numNodes, "localhost", serverPort); + + // set up responses + for (int i = 0; i < nodeIds.size(); i++) { + HttpResponse response = new HttpResponse(Status.OK, expectedBody); + response.addHeaders(expectedResponseHeaders); + server.addResponseAction(new HttpResponseAction(response)); + } + + // headers + expectedRequestHeaders.put("Content-Type", "application/xml"); + + Set responses = replicator.replicate( + nodeIds, + method, + prototypeUri, + expectedEntity, + expectedRequestHeaders); + + Set returnedNodeIds = new HashSet<>(); + for (NodeResponse response : responses) { + + // check if we received an exception + if (response.hasThrowable()) { + throw response.getThrowable(); + } + + // gather ids to verify later + returnedNodeIds.add(response.getNodeId()); + + // check status + assertEquals(Status.OK.getStatusCode(), response.getStatus()); + + Response serverResponse = response.getResponse(); + + // check response headers are copied + assertTrue(containsHeaders(expectedResponseHeaders, serverResponse.getMetadata())); + + // check entity stream + assertTrue(isEquals((StreamingOutput) serverResponse.getEntity(), expectedBody)); + + } + + // check node Ids + assertEquals(nodeIds, returnedNodeIds); + } + + private Set createNodes(int num, String host, int apiPort) { + Set result = new HashSet<>(); + for (int i = 0; i < num; i++) { + result.add(new NodeIdentifier(String.valueOf(i), host, apiPort, host, 1)); + } + return result; + } + + private boolean isEquals(StreamingOutput so, String expectedText) throws IOException { + final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + so.write(baos); + return expectedText.equals(new String(baos.toByteArray())); + } + + private boolean containsHeaders(Map expectedHeaders, MultivaluedMap metadata) { + for (Map.Entry expectedEntry : expectedHeaders.entrySet()) { + if (expectedEntry.getValue().equals(metadata.getFirst(expectedEntry.getKey())) == false) { + return false; + } + } + return true; + } + +} + +@XmlRootElement +class Entity { +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java new file mode 100644 index 0000000000..d45a4d1d15 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/HttpResponseMapperImplTest.java @@ -0,0 +1,126 @@ +/* + * 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.cluster.manager.impl; + +import org.apache.nifi.cluster.manager.impl.HttpResponseMapperImpl; +import com.sun.jersey.api.client.ClientResponse; +import com.sun.jersey.core.util.MultivaluedMapImpl; +import java.io.ByteArrayInputStream; +import java.util.Map; +import java.util.HashSet; +import java.util.Set; +import java.net.URI; +import java.net.URISyntaxException; +import org.apache.nifi.cluster.manager.NodeResponse; +import org.apache.nifi.cluster.node.Node; +import org.apache.nifi.cluster.node.Node.Status; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.junit.Before; +import org.junit.Test; +import static org.junit.Assert.*; +import static org.mockito.Mockito.*; + +/** + * @author unattributed + */ +public class HttpResponseMapperImplTest { + + private HttpResponseMapperImpl mapper; + + private URI dummyUri; + + @Before + public void setup() throws URISyntaxException { + mapper = new HttpResponseMapperImpl(); + dummyUri = new URI("http://dummy.com"); + } + + @Test + public void testToNodeStatusWithNo2xxResponses() { + + Set nodeResponses = new HashSet<>(); + nodeResponses.add(createNodeResourceResponse("1", 400)); + nodeResponses.add(createNodeResourceResponse("2", 100)); + nodeResponses.add(createNodeResourceResponse("3", 300)); + nodeResponses.add(createNodeResourceResponse("4", 500)); + + Map map = mapper.map(dummyUri, nodeResponses); + + // since no 2xx responses, any 5xx is disconnected + for (Map.Entry entry : map.entrySet()) { + NodeResponse response = entry.getKey(); + Status status = entry.getValue(); + switch (response.getNodeId().getId()) { + case "1": + assertTrue(status == Node.Status.CONNECTED); + break; + case "2": + assertTrue(status == Node.Status.CONNECTED); + break; + case "3": + assertTrue(status == Node.Status.CONNECTED); + break; + case "4": + assertTrue(status == Node.Status.DISCONNECTED); + break; + } + } + } + + @Test + public void testToNodeStatusWith2xxResponses() { + + Set nodeResponses = new HashSet<>(); + nodeResponses.add(createNodeResourceResponse("1", 200)); + nodeResponses.add(createNodeResourceResponse("2", 100)); + nodeResponses.add(createNodeResourceResponse("3", 300)); + nodeResponses.add(createNodeResourceResponse("4", 500)); + + Map map = mapper.map(dummyUri, nodeResponses); + + // since there were 2xx responses, any non-2xx is disconnected + for (Map.Entry entry : map.entrySet()) { + NodeResponse response = entry.getKey(); + Status status = entry.getValue(); + switch (response.getNodeId().getId()) { + case "1": + assertTrue(status == Node.Status.CONNECTED); + break; + case "2": + assertTrue(status == Node.Status.DISCONNECTED); + break; + case "3": + assertTrue(status == Node.Status.DISCONNECTED); + break; + case "4": + assertTrue(status == Node.Status.DISCONNECTED); + break; + } + } + } + + private NodeResponse createNodeResourceResponse(String nodeId, int statusCode) { + + ClientResponse clientResponse = mock(ClientResponse.class); + when(clientResponse.getStatus()).thenReturn(statusCode); + when(clientResponse.getHeaders()).thenReturn(new MultivaluedMapImpl()); + when(clientResponse.getEntityInputStream()).thenReturn(new ByteArrayInputStream(new byte[0])); + + NodeIdentifier nodeIdentifier = new NodeIdentifier(nodeId, "localhost", 1, "localhost", 1); + return new NodeResponse(nodeIdentifier, "GET", dummyUri, clientResponse, 1L, "111"); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java new file mode 100644 index 0000000000..7347a94958 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/impl/TestWebClusterManager.java @@ -0,0 +1,52 @@ +/* + * 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.cluster.manager.impl; + +import org.apache.nifi.cluster.manager.impl.WebClusterManager; +import static org.junit.Assert.assertEquals; + +import java.text.DateFormat; +import java.text.ParseException; +import java.text.SimpleDateFormat; +import java.util.Date; + +import org.junit.Test; + +public class TestWebClusterManager { + + @Test + public void testNormalizedStatusSnapshotDate() throws ParseException { + final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:SS.SSS"); + final Date date1 = df.parse("2014/01/01 00:00:00.000"); + final Date date2 = df.parse("2014/01/01 00:04:59.999"); + final Date date3 = df.parse("2014/01/01 00:05:00.000"); + final Date date4 = df.parse("2014/01/01 00:05:00.001"); + + final Date normalized1 = WebClusterManager.normalizeStatusSnapshotDate(date1, 300000); + assertEquals(date1, normalized1); + + final Date normalized2 = WebClusterManager.normalizeStatusSnapshotDate(date2, 300000); + assertEquals(date1, normalized2); + + final Date normalized3 = WebClusterManager.normalizeStatusSnapshotDate(date3, 300000); + assertEquals(date3, normalized3); + + final Date normalized4 = WebClusterManager.normalizeStatusSnapshotDate(date4, 300000); + assertEquals(date3, normalized4); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java new file mode 100644 index 0000000000..35380ddb57 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpRequest.java @@ -0,0 +1,239 @@ +/* + * 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.cluster.manager.testutils; + +import java.io.IOException; +import java.io.Reader; +import java.io.UnsupportedEncodingException; +import java.net.URLDecoder; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import javax.ws.rs.HttpMethod; +import javax.ws.rs.core.MediaType; +import org.apache.commons.lang3.StringUtils; + +/** + * Encapsulates an HTTP request. The toString method returns the + * specification-compliant request. + * + * @author unattributed + */ +public class HttpRequest { + + private String method; + private String uri; + private String rawUri; + private String version; + private String body; + private String rawRequest; + private Map headers = new HashMap<>(); + private Map> parameters = new HashMap<>(); + + public static HttpRequestBuilder createFromRequestLine(final String requestLine) { + return new HttpRequestBuilder(requestLine); + } + + public String getBody() { + return body; + } + + public Map getHeaders() { + return Collections.unmodifiableMap(headers); + } + + public String getHeaderValue(final String header) { + for (final Map.Entry entry : getHeaders().entrySet()) { + if (entry.getKey().equalsIgnoreCase(header)) { + return entry.getValue(); + } + } + return null; + } + + public String getMethod() { + return method; + } + + public Map> getParameters() { + final Map> result = new HashMap<>(); + for (final Map.Entry> entry : parameters.entrySet()) { + result.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); + } + return Collections.unmodifiableMap(result); + } + + public String getUri() { + return uri; + } + + public String getRawUri() { + return rawUri; + } + + public String getVersion() { + return version; + } + + @Override + public String toString() { + return rawRequest; + } + + /** + * A builder for constructing basic HTTP requests. It handles only enough of + * the HTTP specification to support basic unit testing, and it should not + * be used otherwise. + */ + public static class HttpRequestBuilder { + + private String method; + private String uri; + private String rawUri; + private String version; + private Map headers = new HashMap<>(); + private Map> parameters = new HashMap<>(); + private int contentLength = 0; + private String contentType; + private String body = ""; + private StringBuilder rawRequest = new StringBuilder(); + + private HttpRequestBuilder(final String requestLine) { + + final String[] tokens = requestLine.split(" "); + if (tokens.length != 3) { + throw new IllegalArgumentException("Invalid HTTP Request Line: " + requestLine); + } + + method = tokens[0]; + if (HttpMethod.GET.equalsIgnoreCase(method) || HttpMethod.HEAD.equalsIgnoreCase(method) || HttpMethod.DELETE.equalsIgnoreCase(method) || HttpMethod.OPTIONS.equalsIgnoreCase(method)) { + final int queryIndex = tokens[1].indexOf("?"); + if (queryIndex > -1) { + uri = tokens[1].substring(0, queryIndex); + addParameters(tokens[1].substring(queryIndex + 1)); + } else { + uri = tokens[1]; + } + } + rawUri = tokens[1]; + version = tokens[2]; + rawRequest.append(requestLine).append("\n"); + } + + private void addHeader(final String key, final String value) { + if (key.contains(" ")) { + throw new IllegalArgumentException("Header key may not contain spaces."); + } else if ("content-length".equalsIgnoreCase(key)) { + contentLength = (StringUtils.isBlank(value.trim())) ? 0 : Integer.parseInt(value.trim()); + } else if ("content-type".equalsIgnoreCase(key)) { + contentType = value.trim(); + } + headers.put(key, value); + } + + public void addHeader(final String header) { + final int firstColonIndex = header.indexOf(":"); + if (firstColonIndex < 0) { + throw new IllegalArgumentException("Invalid HTTP Header line: " + header); + } + addHeader(header.substring(0, firstColonIndex), header.substring(firstColonIndex + 1)); + rawRequest.append(header).append("\n"); + } + + // final because constructor calls it + public final void addParameters(final String queryString) { + + if (StringUtils.isBlank(queryString)) { + return; + } + + final String normQueryString; + if (queryString.startsWith("?")) { + normQueryString = queryString.substring(1); + } else { + normQueryString = queryString; + } + final String[] keyValuePairs = normQueryString.split("&"); + for (final String keyValuePair : keyValuePairs) { + final String[] keyValueTokens = keyValuePair.split("="); + try { + addParameter( + URLDecoder.decode(keyValueTokens[0], "utf-8"), + URLDecoder.decode(keyValueTokens[1], "utf-8") + ); + } catch (UnsupportedEncodingException use) { + throw new RuntimeException(use); + } + } + } + + public void addParameter(final String key, final String value) { + + if (key.contains(" ")) { + throw new IllegalArgumentException("Parameter key may not contain spaces: " + key); + } + + final List values; + if (parameters.containsKey(key)) { + values = parameters.get(key); + } else { + values = new ArrayList<>(); + parameters.put(key, values); + } + values.add(value); + } + + public void addBody(final Reader reader) throws IOException { + + if (contentLength <= 0) { + return; + } + + final char[] buf = new char[contentLength]; + int offset = 0; + int numRead = 0; + while (offset < buf.length && (numRead = reader.read(buf, offset, buf.length - offset)) >= 0) { + offset += numRead; + } + body = new String(buf); + rawRequest.append("\n"); + rawRequest.append(body); + } + + public HttpRequest build() throws UnsupportedEncodingException { + + if (HttpMethod.GET.equalsIgnoreCase(method) == false && HttpMethod.HEAD.equalsIgnoreCase(method) == false && contentType.equalsIgnoreCase(MediaType.APPLICATION_FORM_URLENCODED)) { + addParameters(body); + } + + final HttpRequest request = new HttpRequest(); + request.method = this.method; + request.uri = this.uri; + request.rawUri = this.rawUri; + request.version = this.version; + request.headers.putAll(this.headers); + request.parameters.putAll(this.parameters); + request.body = this.body; + request.rawRequest = this.rawRequest.toString(); + + return request; + } + + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java new file mode 100644 index 0000000000..3aa29318a3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponse.java @@ -0,0 +1,93 @@ +/* + * 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.cluster.manager.testutils; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import javax.ws.rs.core.Response.Status; + +/** + * Encapsulates an HTTP response. The toString method returns the + * specification-compliant response. + * + * @author unattributed + */ +public class HttpResponse { + + private final Status status; + private final String entity; + private final Map headers = new HashMap<>(); + + public HttpResponse(final Status status, final String entity) { + this.status = status; + this.entity = entity; + headers.put("content-length", String.valueOf(entity.getBytes().length)); + } + + public String getEntity() { + return entity; + } + + public Status getStatus() { + return status; + } + + public Map getHeaders() { + return Collections.unmodifiableMap(headers); + } + + public void addHeader(final String key, final String value) { + if (key.contains(" ")) { + throw new IllegalArgumentException("Header key may not contain spaces."); + } else if ("content-length".equalsIgnoreCase(key)) { + throw new IllegalArgumentException("Content-Length header is set automatically based on length of content."); + } + headers.put(key, value); + } + + public void addHeaders(final Map headers) { + for (final Map.Entry entry : headers.entrySet()) { + addHeader(entry.getKey(), entry.getValue()); + } + } + + @Override + public String toString() { + + final StringBuilder strb = new StringBuilder(); + + // response line + strb.append("HTTP/1.1 ") + .append(status.getStatusCode()) + .append(" ") + .append(status.getReasonPhrase()) + .append("\n"); + + // headers + for (final Map.Entry entry : headers.entrySet()) { + strb.append(entry.getKey()).append(": ").append(entry.getValue()).append("\n"); + } + + strb.append("\n"); + + // body + strb.append(entity); + + return strb.toString(); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java new file mode 100644 index 0000000000..28615d0348 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpResponseAction.java @@ -0,0 +1,60 @@ +/* + * 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.cluster.manager.testutils; + +/** + * Wraps a HttpResponse with a time-delay. When the action is applied, the + * currently executing thread sleeps for the given delay before returning the + * response to the caller. + * + * This class is good for simulating network latency. + * + * @author unattributed + */ +public class HttpResponseAction { + + private final HttpResponse response; + + private final int waitTimeMs; + + public HttpResponseAction(final HttpResponse response) { + this(response, 0); + } + + public HttpResponseAction(final HttpResponse response, final int waitTimeMs) { + this.response = response; + this.waitTimeMs = waitTimeMs; + } + + public HttpResponse apply() { + try { + Thread.sleep(waitTimeMs); + } catch (final InterruptedException ie) { + throw new RuntimeException(ie); + } + + return response; + } + + public HttpResponse getResponse() { + return response; + } + + public int getWaitTimeMs() { + return waitTimeMs; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java new file mode 100644 index 0000000000..f17a66c3eb --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/manager/testutils/HttpServer.java @@ -0,0 +1,240 @@ +/* + * 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.cluster.manager.testutils; + +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.PrintWriter; +import java.io.Reader; +import java.net.ServerSocket; +import java.net.Socket; +import java.net.SocketException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.apache.nifi.cluster.manager.testutils.HttpRequest.HttpRequestBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A simple HTTP web server that allows clients to register canned-responses to + * respond to received requests. + * + * @author unattributed + */ +public class HttpServer { + + private static final Logger logger = LoggerFactory.getLogger(HttpServer.class); + + private final ExecutorService executorService; + private final ServerSocket serverSocket; + private final Queue responseQueue = new ConcurrentLinkedQueue<>(); + private final Map checkedHeaders = new HashMap<>(); + private final Map> checkedParameters = new HashMap<>(); + private final int port; + + public HttpServer(int numThreads, int port) throws IOException { + this.port = port; + executorService = Executors.newFixedThreadPool(numThreads); + serverSocket = new ServerSocket(port); + } + + public void start() { + + new Thread() { + @Override + public void run() { + while (isRunning()) { + try { + final Socket conn = serverSocket.accept(); + executorService.execute(new Runnable() { + @Override + public void run() { + handleRequest(conn); + if (conn.isClosed() == false) { + try { + conn.close(); + } catch (IOException ioe) { + } + } + } + }); + } catch (final SocketException se) { + /* ignored */ + } catch (final IOException ioe) { + if (logger.isDebugEnabled()) { + logger.warn("", ioe); + } + } + } + } + ; + } + + .start(); + } + + public boolean isRunning() { + return executorService.isShutdown() == false; + } + + public void stop() { + // shutdown server socket + try { + if (serverSocket.isClosed() == false) { + serverSocket.close(); + } + } catch (final Exception ex) { + throw new RuntimeException(ex); + } + + // shutdown executor service + try { + executorService.shutdown(); + executorService.awaitTermination(3, TimeUnit.SECONDS); + } catch (final Exception ex) { + throw new RuntimeException(ex); + } + } + + public int getPort() { + if (isRunning()) { + return serverSocket.getLocalPort(); + } else { + return port; + } + } + + public Queue addResponseAction(final HttpResponseAction response) { + responseQueue.add(response); + return responseQueue; + } + + public void addCheckedHeaders(final Map headers) { + checkedHeaders.putAll(headers); + } + + public void addCheckedParameters(final Map> parameters) { + checkedParameters.putAll(parameters); + } + + private void handleRequest(final Socket conn) { + try { + + final HttpRequest httpRequest = buildRequest(conn.getInputStream()); + + if (logger.isDebugEnabled()) { + logger.debug("\n" + httpRequest); + } + + // check headers + final Map reqHeaders = httpRequest.getHeaders(); + for (final Map.Entry entry : checkedHeaders.entrySet()) { + if (reqHeaders.containsKey(entry.getKey())) { + if (entry.getValue().equals(reqHeaders.get(entry.getKey()))) { + logger.error("Incorrect HTTP request header value received for checked header: " + entry.getKey()); + conn.close(); + return; + } + } else { + logger.error("Missing checked header: " + entry.getKey()); + conn.close(); + return; + } + } + + // check parameters + final Map> reqParams = httpRequest.getParameters(); + for (final Map.Entry> entry : checkedParameters.entrySet()) { + if (reqParams.containsKey(entry.getKey())) { + if (entry.getValue().equals(reqParams.get(entry.getKey())) == false) { + logger.error("Incorrect HTTP request parameter values received for checked parameter: " + entry.getKey()); + conn.close(); + return; + } + } else { + logger.error("Missing checked parameter: " + entry.getKey()); + conn.close(); + return; + } + } + + // apply the next response + final HttpResponseAction response = responseQueue.remove(); + response.apply(); + + // send the response to client + final PrintWriter pw = new PrintWriter(conn.getOutputStream(), true); + + if (logger.isDebugEnabled()) { + logger.debug("\n" + response.getResponse()); + } + + pw.print(response.getResponse()); + pw.flush(); + + } catch (IOException ioe) { /* ignored */ } + } + + private HttpRequest buildRequest(final InputStream requestIs) throws IOException { + return new HttpRequestReader().read(new InputStreamReader(requestIs)); + } + + // reads an HTTP request from the given reader + private class HttpRequestReader { + + public HttpRequest read(final Reader reader) throws IOException { + + HttpRequestBuilder builder = null; + String line = ""; + boolean isRequestLine = true; + while ((line = readLine(reader)).isEmpty() == false) { + if (isRequestLine) { + builder = HttpRequest.createFromRequestLine(line); + isRequestLine = false; + } else { + builder.addHeader(line); + } + } + + if (builder != null) { + builder.addBody(reader); + } + + return builder.build(); + } + + private String readLine(final Reader reader) throws IOException { + + /* read character at time to prevent blocking */ + final StringBuilder strb = new StringBuilder(); + char c; + while ((c = (char) reader.read()) != '\n') { + if (c != '\r') { + strb.append(c); + } + } + return strb.toString(); + } + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java new file mode 100644 index 0000000000..96943c2462 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterManagerProtocolSenderImplTest.java @@ -0,0 +1,133 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.net.InetAddress; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.FlowRequestMessage; +import org.apache.nifi.cluster.protocol.message.FlowResponseMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.*; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * @author unattributed + */ +public class ClusterManagerProtocolSenderImplTest { + + private InetAddress address; + + private int port; + + private SocketProtocolListener listener; + + private ClusterManagerProtocolSenderImpl sender; + + private ProtocolHandler mockHandler; + + @Before + public void setup() throws IOException { + + address = InetAddress.getLocalHost(); + ServerSocketConfiguration serverSocketConfiguration = new ServerSocketConfiguration(); + + mockHandler = mock(ProtocolHandler.class); + + ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext); + listener.addHandler(mockHandler); + listener.start(); + + port = listener.getPort(); + + SocketConfiguration socketConfiguration = new SocketConfiguration(); + sender = new ClusterManagerProtocolSenderImpl(socketConfiguration, protocolContext); + } + + @After + public void teardown() throws IOException { + if (listener.isRunning()) { + listener.stop(); + } + } + + @Test + public void testRequestFlow() throws Exception { + + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new FlowResponseMessage()); + FlowRequestMessage request = new FlowRequestMessage(); + request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port)); + FlowResponseMessage response = sender.requestFlow(request); + assertNotNull(response); + } + + @Test + public void testRequestFlowWithBadResponseMessage() throws Exception { + + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage()); + FlowRequestMessage request = new FlowRequestMessage(); + request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port)); + try { + sender.requestFlow(request); + fail("failed to throw exception"); + } catch (ProtocolException pe) { + } + + } + + @Test + public void testRequestFlowDelayedResponse() throws Exception { + + final int time = 250; + sender.getSocketConfiguration().setSocketTimeout(time); + + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() { + @Override + public FlowResponseMessage answer(InvocationOnMock invocation) throws Throwable { + Thread.sleep(time * 3); + return new FlowResponseMessage(); + } + }); + FlowRequestMessage request = new FlowRequestMessage(); + request.setNodeId(new NodeIdentifier("id", "api-address", 1, address.getHostAddress(), port)); + try { + sender.requestFlow(request); + fail("failed to throw exception"); + } catch (ProtocolException pe) { + } + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java new file mode 100644 index 0000000000..4a6957105e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServiceLocatorTest.java @@ -0,0 +1,119 @@ +/* + * 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.cluster.protocol.impl; + +import java.net.InetSocketAddress; +import java.util.concurrent.TimeUnit; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Test; +import static org.mockito.Mockito.*; +import org.mockito.stubbing.OngoingStubbing; + +public class ClusterServiceLocatorTest { + + private ClusterServiceDiscovery mockServiceDiscovery; + + private int fixedPort; + + private DiscoverableService fixedService; + + private ClusterServiceLocator serviceDiscoveryLocator; + + private ClusterServiceLocator serviceDiscoveryFixedPortLocator; + + private ClusterServiceLocator fixedServiceLocator; + + @Before + public void setup() throws Exception { + + fixedPort = 1; + mockServiceDiscovery = mock(ClusterServiceDiscovery.class); + fixedService = new DiscoverableServiceImpl("some-service", InetSocketAddress.createUnresolved("some-host", 20)); + + serviceDiscoveryLocator = new ClusterServiceLocator(mockServiceDiscovery); + serviceDiscoveryFixedPortLocator = new ClusterServiceLocator(mockServiceDiscovery, fixedPort); + fixedServiceLocator = new ClusterServiceLocator(fixedService); + + } + + @Test + public void getServiceWhenServiceDiscoveryNotStarted() { + assertNull(serviceDiscoveryLocator.getService()); + } + + @Test + public void getServiceWhenServiceDiscoveryFixedPortNotStarted() { + assertNull(serviceDiscoveryLocator.getService()); + } + + @Test + public void getServiceWhenFixedServiceNotStarted() { + assertEquals(fixedService, fixedServiceLocator.getService()); + } + + @Test + public void getServiceNotOnFirstAttempt() { + + ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); + config.setNumAttempts(2); + config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); + config.setTimeBetweenAttempts(1); + + serviceDiscoveryLocator.setAttemptsConfig(config); + + OngoingStubbing stubbing = null; + for (int i = 0; i < config.getNumAttempts() - 1; i++) { + if (stubbing == null) { + stubbing = when(mockServiceDiscovery.getService()).thenReturn(null); + } else { + stubbing.thenReturn(null); + } + } + stubbing.thenReturn(fixedService); + + assertEquals(fixedService, serviceDiscoveryLocator.getService()); + + } + + @Test + public void getServiceNotOnFirstAttemptWithFixedPort() { + + ClusterServiceLocator.AttemptsConfig config = new ClusterServiceLocator.AttemptsConfig(); + config.setNumAttempts(2); + config.setTimeBetweenAttempsUnit(TimeUnit.SECONDS); + config.setTimeBetweenAttempts(1); + + serviceDiscoveryFixedPortLocator.setAttemptsConfig(config); + + OngoingStubbing stubbing = null; + for (int i = 0; i < config.getNumAttempts() - 1; i++) { + if (stubbing == null) { + stubbing = when(mockServiceDiscovery.getService()).thenReturn(null); + } else { + stubbing.thenReturn(null); + } + } + stubbing.thenReturn(fixedService); + + InetSocketAddress resultAddress = InetSocketAddress.createUnresolved(fixedService.getServiceAddress().getHostName(), fixedPort); + DiscoverableService resultService = new DiscoverableServiceImpl(fixedService.getServiceName(), resultAddress); + assertEquals(resultService, serviceDiscoveryFixedPortLocator.getService()); + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java new file mode 100644 index 0000000000..4d85d1a130 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/ClusterServicesBroadcasterTest.java @@ -0,0 +1,131 @@ +/* + * 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.cluster.protocol.impl; + +import java.net.InetSocketAddress; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.message.ServiceBroadcastMessage; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +/** + * @author unattributed + */ +public class ClusterServicesBroadcasterTest { + + private ClusterServicesBroadcaster broadcaster; + + private MulticastProtocolListener listener; + + private DummyProtocolHandler handler; + + private InetSocketAddress multicastAddress; + + private DiscoverableService broadcastedService; + + private ProtocolContext protocolContext; + + private MulticastConfiguration configuration; + + @Before + public void setup() throws Exception { + + broadcastedService = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", 11111)); + + multicastAddress = new InetSocketAddress("225.1.1.1", 22222); + + configuration = new MulticastConfiguration(); + + protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + broadcaster = new ClusterServicesBroadcaster(multicastAddress, configuration, protocolContext, "500 ms"); + broadcaster.addService(broadcastedService); + + handler = new DummyProtocolHandler(); + listener = new MulticastProtocolListener(5, multicastAddress, configuration, protocolContext); + listener.addHandler(handler); + } + + @After + public void teardown() { + + if (broadcaster.isRunning()) { + broadcaster.stop(); + } + + try { + if (listener.isRunning()) { + listener.stop(); + } + } catch (Exception ex) { + ex.printStackTrace(System.out); + } + + } + + @Test + @Ignore + public void testBroadcastReceived() throws Exception { + + broadcaster.start(); + listener.start(); + + Thread.sleep(1000); + + listener.stop(); + + assertNotNull(handler.getProtocolMessage()); + assertEquals(ProtocolMessage.MessageType.SERVICE_BROADCAST, handler.getProtocolMessage().getType()); + final ServiceBroadcastMessage msg = (ServiceBroadcastMessage) handler.getProtocolMessage(); + assertEquals(broadcastedService.getServiceName(), msg.getServiceName()); + assertEquals(broadcastedService.getServiceAddress().getHostName(), msg.getAddress()); + assertEquals(broadcastedService.getServiceAddress().getPort(), msg.getPort()); + } + + private class DummyProtocolHandler implements ProtocolHandler { + + private ProtocolMessage protocolMessage; + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + this.protocolMessage = msg; + return null; + } + + public ProtocolMessage getProtocolMessage() { + return protocolMessage; + } + + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java new file mode 100644 index 0000000000..acd21e88db --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/MulticastProtocolListenerTest.java @@ -0,0 +1,170 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.DatagramPacket; +import java.net.InetSocketAddress; +import java.net.MulticastSocket; +import java.util.ArrayList; +import java.util.List; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.MulticastProtocolMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.multicast.MulticastConfiguration; +import org.apache.nifi.io.socket.multicast.MulticastUtils; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +/** + * @author unattributed + */ +public class MulticastProtocolListenerTest { + + private MulticastProtocolListener listener; + + private MulticastSocket socket; + + private InetSocketAddress address; + + private MulticastConfiguration configuration; + + private ProtocolContext protocolContext; + + @Before + public void setup() throws Exception { + + address = new InetSocketAddress("226.1.1.1", 60000); + configuration = new MulticastConfiguration(); + + protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + listener = new MulticastProtocolListener(5, address, configuration, protocolContext); + listener.start(); + + socket = MulticastUtils.createMulticastSocket(address.getPort(), configuration); + } + + @After + public void teardown() throws IOException { + try { + if (listener.isRunning()) { + listener.stop(); + } + } finally { + MulticastUtils.closeQuietly(socket); + } + } + + @Test + public void testBadRequest() throws Exception { + DelayedProtocolHandler handler = new DelayedProtocolHandler(0); + listener.addHandler(handler); + DatagramPacket packet = new DatagramPacket(new byte[]{5}, 1, address); + socket.send(packet); + Thread.sleep(250); + assertEquals(0, handler.getMessages().size()); + } + + @Test + @Ignore + public void testRequest() throws Exception { + + ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler(); + listener.addHandler(handler); + + ProtocolMessage msg = new PingMessage(); + MulticastProtocolMessage multicastMsg = new MulticastProtocolMessage("some-id", msg); + + // marshal message to output stream + ProtocolMessageMarshaller marshaller = protocolContext.createMarshaller(); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + marshaller.marshal(multicastMsg, baos); + byte[] requestPacketBytes = baos.toByteArray(); + DatagramPacket packet = new DatagramPacket(requestPacketBytes, requestPacketBytes.length, address); + socket.send(packet); + + Thread.sleep(250); + assertEquals(1, handler.getMessages().size()); + assertEquals(msg.getType(), handler.getMessages().get(0).getType()); + + } + + private class ReflexiveProtocolHandler implements ProtocolHandler { + + private List messages = new ArrayList<>(); + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + messages.add(msg); + return msg; + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } + + } + + private class DelayedProtocolHandler implements ProtocolHandler { + + private int delay = 0; + + private List messages = new ArrayList<>(); + + public DelayedProtocolHandler(int delay) { + this.delay = delay; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + try { + messages.add(msg); + Thread.sleep(delay); + return null; + } catch (final InterruptedException ie) { + throw new ProtocolException(ie); + } + + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } + + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java new file mode 100644 index 0000000000..7c62d2f984 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/NodeProtocolSenderImplTest.java @@ -0,0 +1,201 @@ +/* + * 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.cluster.protocol.impl; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.UUID; + +import org.apache.nifi.cluster.HeartbeatPayload; +import org.apache.nifi.cluster.protocol.ConnectionRequest; +import org.apache.nifi.cluster.protocol.ConnectionResponse; +import org.apache.nifi.cluster.protocol.Heartbeat; +import org.apache.nifi.cluster.protocol.NodeIdentifier; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.StandardDataFlow; +import org.apache.nifi.cluster.protocol.UnknownServiceAddressException; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.ConnectionRequestMessage; +import org.apache.nifi.cluster.protocol.message.ConnectionResponseMessage; +import org.apache.nifi.cluster.protocol.message.ControllerStartupFailureMessage; +import org.apache.nifi.cluster.protocol.message.HeartbeatMessage; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.io.socket.multicast.DiscoverableService; +import org.apache.nifi.io.socket.multicast.DiscoverableServiceImpl; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + +/** + * @author unattributed + */ +public class NodeProtocolSenderImplTest { + + private SocketProtocolListener listener; + + private NodeProtocolSenderImpl sender; + + private DiscoverableService service; + + private ServerSocketConfiguration serverSocketConfiguration; + + private ClusterServiceLocator mockServiceLocator; + + private ProtocolHandler mockHandler; + + private NodeIdentifier nodeIdentifier; + + @Before + public void setup() throws IOException { + + serverSocketConfiguration = new ServerSocketConfiguration(); + + mockServiceLocator = mock(ClusterServiceLocator.class); + mockHandler = mock(ProtocolHandler.class); + + nodeIdentifier = new NodeIdentifier("1", "localhost", 1234, "localhost", 5678); + + ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + + listener = new SocketProtocolListener(5, 0, serverSocketConfiguration, protocolContext); + listener.setShutdownListenerSeconds(3); + listener.addHandler(mockHandler); + listener.start(); + + service = new DiscoverableServiceImpl("some-service", new InetSocketAddress("localhost", listener.getPort())); + + SocketConfiguration socketConfiguration = new SocketConfiguration(); + socketConfiguration.setReuseAddress(true); + sender = new NodeProtocolSenderImpl(mockServiceLocator, socketConfiguration, protocolContext); + } + + @After + public void teardown() throws IOException { + if (listener.isRunning()) { + listener.stop(); + } + } + + @Test + public void testConnect() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + ConnectionResponseMessage mockMessage = new ConnectionResponseMessage(); + mockMessage.setConnectionResponse(new ConnectionResponse(nodeIdentifier, new StandardDataFlow("flow".getBytes("UTF-8"), new byte[0], new byte[0]), false, null, null, UUID.randomUUID().toString())); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(mockMessage); + + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + ConnectionResponseMessage response = sender.requestConnection(request); + assertNotNull(response); + } + + @Test(expected = UnknownServiceAddressException.class) + public void testConnectNoClusterManagerAddress() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(null); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new ConnectionResponseMessage()); + + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + + sender.requestConnection(request); + fail("failed to throw exception"); + } + + @Test(expected = ProtocolException.class) + public void testConnectBadResponse() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(new PingMessage()); + + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + + sender.requestConnection(request); + fail("failed to throw exception"); + + } + + @Test(expected = ProtocolException.class) + public void testConnectDelayedResponse() throws Exception { + + final int time = 250; + sender.getSocketConfiguration().setSocketTimeout(time); + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenAnswer(new Answer() { + @Override + public ConnectionResponseMessage answer(InvocationOnMock invocation) throws Throwable { + Thread.sleep(time * 3); + return new ConnectionResponseMessage(); + } + }); + ConnectionRequestMessage request = new ConnectionRequestMessage(); + request.setConnectionRequest(new ConnectionRequest(nodeIdentifier)); + + sender.requestConnection(request); + fail("failed to throw exception"); + + } + + @Test + public void testHeartbeat() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null); + + HeartbeatMessage msg = new HeartbeatMessage(); + HeartbeatPayload hbPayload = new HeartbeatPayload(); + Heartbeat hb = new Heartbeat(new NodeIdentifier("id", "localhost", 3, "localhost", 4), false, false, hbPayload.marshal()); + msg.setHeartbeat(hb); + sender.heartbeat(msg); + } + + @Test + public void testNotifyControllerStartupFailure() throws Exception { + + when(mockServiceLocator.getService()).thenReturn(service); + when(mockHandler.canHandle(any(ProtocolMessage.class))).thenReturn(Boolean.TRUE); + when(mockHandler.handle(any(ProtocolMessage.class))).thenReturn(null); + + ControllerStartupFailureMessage msg = new ControllerStartupFailureMessage(); + msg.setNodeId(new NodeIdentifier("some-id", "some-addr", 1, "some-addr", 1)); + msg.setExceptionMessage("some exception"); + sender.notifyControllerStartupFailure(msg); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java new file mode 100644 index 0000000000..92a7d2a943 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/impl/SocketProtocolListenerTest.java @@ -0,0 +1,132 @@ +/* + * 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.cluster.protocol.impl; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.net.SocketTimeoutException; +import org.apache.nifi.cluster.protocol.ProtocolContext; +import org.apache.nifi.cluster.protocol.ProtocolMessageMarshaller; +import org.apache.nifi.cluster.protocol.ProtocolMessageUnmarshaller; +import org.apache.nifi.cluster.protocol.jaxb.JaxbProtocolContext; +import org.apache.nifi.cluster.protocol.jaxb.message.JaxbProtocolUtils; +import org.apache.nifi.cluster.protocol.message.PingMessage; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; +import org.apache.nifi.cluster.protocol.testutils.DelayedProtocolHandler; +import org.apache.nifi.cluster.protocol.testutils.ReflexiveProtocolHandler; +import org.apache.nifi.io.socket.ServerSocketConfiguration; +import org.apache.nifi.io.socket.SocketConfiguration; +import org.apache.nifi.io.socket.SocketUtils; +import org.junit.After; +import static org.junit.Assert.*; +import org.junit.Before; +import org.junit.Test; + +/** + * @author unattributed + */ +public class SocketProtocolListenerTest { + + private SocketProtocolListener listener; + + private Socket socket; + + private ProtocolMessageMarshaller marshaller; + + private ProtocolMessageUnmarshaller unmarshaller; + + @Before + public void setup() throws Exception { + + final ProtocolContext protocolContext = new JaxbProtocolContext(JaxbProtocolUtils.JAXB_CONTEXT); + marshaller = protocolContext.createMarshaller(); + unmarshaller = protocolContext.createUnmarshaller(); + + ServerSocketConfiguration configuration = new ServerSocketConfiguration(); + configuration.setSocketTimeout(1000); + + listener = new SocketProtocolListener(5, 0, configuration, protocolContext); + listener.start(); + + int port = listener.getPort(); + + SocketConfiguration config = new SocketConfiguration(); + config.setReuseAddress(true); + config.setSocketTimeout(1000); + socket = SocketUtils.createSocket(new InetSocketAddress("localhost", port), config); + } + + @After + public void teardown() throws IOException { + try { + if (listener.isRunning()) { + listener.stop(); + } + } finally { + SocketUtils.closeQuietly(socket); + } + } + + @Test + public void testBadRequest() throws Exception { + DelayedProtocolHandler handler = new DelayedProtocolHandler(0); + listener.addHandler(handler); + socket.getOutputStream().write(5); + Thread.sleep(250); + assertEquals(0, handler.getMessages().size()); + } + + @Test + public void testRequest() throws Exception { + ProtocolMessage msg = new PingMessage(); + + ReflexiveProtocolHandler handler = new ReflexiveProtocolHandler(); + listener.addHandler(handler); + + // marshal message to output stream + marshaller.marshal(msg, socket.getOutputStream()); + + // unmarshall response and return + ProtocolMessage response = unmarshaller.unmarshal(socket.getInputStream()); + assertEquals(msg.getType(), response.getType()); + + assertEquals(1, handler.getMessages().size()); + assertEquals(msg.getType(), handler.getMessages().get(0).getType()); + } + + @Test + public void testDelayedRequest() throws Exception { + ProtocolMessage msg = new PingMessage(); + + DelayedProtocolHandler handler = new DelayedProtocolHandler(2000); + listener.addHandler(handler); + + // marshal message to output stream + marshaller.marshal(msg, socket.getOutputStream()); + + try { + socket.getInputStream().read(); + fail("Socket timeout not received."); + } catch (SocketTimeoutException ste) { + } + + assertEquals(1, handler.getMessages().size()); + assertEquals(msg.getType(), handler.getMessages().get(0).getType()); + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.java new file mode 100644 index 0000000000..2f16777fae --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/DelayedProtocolHandler.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.cluster.protocol.testutils; + +import java.util.ArrayList; +import java.util.List; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; + +/** + * @author unattributed + */ +public class DelayedProtocolHandler implements ProtocolHandler { + + private int delay = 0; + private List messages = new ArrayList<>(); + + public DelayedProtocolHandler(int delay) { + this.delay = delay; + } + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + try { + messages.add(msg); + Thread.sleep(delay); + return null; + } catch (final InterruptedException ie) { + throw new ProtocolException(ie); + } + + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java new file mode 100644 index 0000000000..e80f52ce3d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/java/org/apache/nifi/cluster/protocol/testutils/ReflexiveProtocolHandler.java @@ -0,0 +1,47 @@ +/* + * 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.cluster.protocol.testutils; + +import java.util.ArrayList; +import java.util.List; +import org.apache.nifi.cluster.protocol.ProtocolException; +import org.apache.nifi.cluster.protocol.ProtocolHandler; +import org.apache.nifi.cluster.protocol.message.ProtocolMessage; + +/** + * @author unattributed + */ +public class ReflexiveProtocolHandler implements ProtocolHandler { + + private List messages = new ArrayList<>(); + + @Override + public ProtocolMessage handle(ProtocolMessage msg) throws ProtocolException { + messages.add(msg); + return msg; + } + + @Override + public boolean canHandle(ProtocolMessage msg) { + return true; + } + + public List getMessages() { + return messages; + } + +} diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml new file mode 100644 index 0000000000..92eb78cbbf --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/logback-test.xml @@ -0,0 +1,48 @@ + + + + + + + %-4r [%t] %-5p %c - %m%n + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/empty.txt new file mode 100755 index 0000000000..e69de29bb2 diff --git a/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt new file mode 100755 index 0000000000..e8e4c2bcc0 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/cluster/src/test/resources/org/apache/nifi/cluster/firewall/impl/ips.txt @@ -0,0 +1,12 @@ + +bad data should be skipped + +# this is a comment + 2.2.2.2 # this is another comment #### +3.3.3.3/8 + +4.4.4.4/24 + +5.5.5.255/31 + +more bad data \ No newline at end of file diff --git a/nar-bundles/framework-bundle/framework/core-api/.gitignore b/nar-bundles/framework-bundle/framework/core-api/.gitignore new file mode 100755 index 0000000000..ea8c4bf7f3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/.gitignore @@ -0,0 +1 @@ +/target diff --git a/nar-bundles/framework-bundle/framework/core-api/pom.xml b/nar-bundles/framework-bundle/framework/core-api/pom.xml new file mode 100644 index 0000000000..b163cd0396 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + org.apache.nifi + nifi-framework-parent + 0.0.1-SNAPSHOT + + core-api + 0.0.1-SNAPSHOT + NiFi Core API + + + org.apache.nifi + nifi-nar + + + org.apache.nifi + remote-communications-utils + + + org.apache.nifi + nifi-runtime + + + org.apache.nifi + client-dto + + + org.apache.nifi + nifi-api + + + org.apache.commons + commons-lang3 + + + org.apache.nifi + nifi-utils + + + org.quartz-scheduler + quartz + + + diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java new file mode 100644 index 0000000000..0092f7a5d5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/AdaptedNodeInformation.java @@ -0,0 +1,66 @@ +/* + * 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.cluster; + +public class AdaptedNodeInformation { + + private String hostname; + private Integer siteToSitePort; + private int apiPort; + private boolean isSiteToSiteSecure; + private int totalFlowFiles; + + public String getHostname() { + return hostname; + } + + public void setHostname(String hostname) { + this.hostname = hostname; + } + + public Integer getSiteToSitePort() { + return siteToSitePort; + } + + public void setSiteToSitePort(Integer siteToSitePort) { + this.siteToSitePort = siteToSitePort; + } + + public int getApiPort() { + return apiPort; + } + + public void setApiPort(int apiPort) { + this.apiPort = apiPort; + } + + public boolean isSiteToSiteSecure() { + return isSiteToSiteSecure; + } + + public void setSiteToSiteSecure(boolean isSiteToSiteSecure) { + this.isSiteToSiteSecure = isSiteToSiteSecure; + } + + public int getTotalFlowFiles() { + return totalFlowFiles; + } + + public void setTotalFlowFiles(int totalFlowFiles) { + this.totalFlowFiles = totalFlowFiles; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java new file mode 100644 index 0000000000..5751c32d39 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/ClusterNodeInformation.java @@ -0,0 +1,67 @@ +/* + * 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.cluster; + +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collection; + +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; +import javax.xml.bind.Unmarshaller; +import javax.xml.bind.annotation.XmlRootElement; +import javax.xml.bind.annotation.adapters.XmlJavaTypeAdapter; + +@XmlRootElement +public class ClusterNodeInformation { + + private Collection nodeInfo; + + private static final JAXBContext JAXB_CONTEXT; + + static { + try { + JAXB_CONTEXT = JAXBContext.newInstance(ClusterNodeInformation.class); + } catch (JAXBException e) { + throw new RuntimeException("Unable to create JAXBContext.", e); + } + } + + public ClusterNodeInformation() { + this.nodeInfo = null; + } + + public void setNodeInformation(final Collection nodeInfo) { + this.nodeInfo = nodeInfo; + } + + @XmlJavaTypeAdapter(NodeInformationAdapter.class) + public Collection getNodeInformation() { + return nodeInfo; + } + + public void marshal(final OutputStream os) throws JAXBException { + final Marshaller marshaller = JAXB_CONTEXT.createMarshaller(); + marshaller.marshal(this, os); + } + + public static ClusterNodeInformation unmarshal(final InputStream is) throws JAXBException { + final Unmarshaller unmarshaller = JAXB_CONTEXT.createUnmarshaller(); + return (ClusterNodeInformation) unmarshaller.unmarshal(is); + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java new file mode 100644 index 0000000000..987ff65cd5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformant.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster; + +public interface NodeInformant { + + ClusterNodeInformation getNodeInformation(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java new file mode 100644 index 0000000000..848eb7e065 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformation.java @@ -0,0 +1,98 @@ +/* + * 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.cluster; + +public class NodeInformation { + + private final String hostname; + private final Integer siteToSitePort; + private final int apiPort; + private final boolean isSiteToSiteSecure; + private final int totalFlowFiles; + + public NodeInformation(final String hostname, final Integer siteToSitePort, final int apiPort, + final boolean isSiteToSiteSecure, final int totalFlowFiles) { + this.hostname = hostname; + this.siteToSitePort = siteToSitePort; + this.apiPort = apiPort; + this.isSiteToSiteSecure = isSiteToSiteSecure; + this.totalFlowFiles = totalFlowFiles; + } + + public String getHostname() { + return hostname; + } + + public int getAPIPort() { + return apiPort; + } + + public Integer getSiteToSitePort() { + return siteToSitePort; + } + + public boolean isSiteToSiteSecure() { + return isSiteToSiteSecure; + } + + public int getTotalFlowFiles() { + return totalFlowFiles; + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof NodeInformation)) { + return false; + } + + final NodeInformation other = (NodeInformation) obj; + if (!hostname.equals(other.hostname)) { + return false; + } + if (siteToSitePort == null && other.siteToSitePort != null) { + return false; + } + if (siteToSitePort != null && other.siteToSitePort == null) { + return false; + } else if (siteToSitePort != null && siteToSitePort.intValue() != other.siteToSitePort.intValue()) { + return false; + } + if (apiPort != other.apiPort) { + return false; + } + if (isSiteToSiteSecure != other.isSiteToSiteSecure) { + return false; + } + return true; + } + + @Override + public int hashCode() { + return 83832 + hostname.hashCode() + (siteToSitePort == null ? 8 : siteToSitePort.hashCode()) + apiPort + (isSiteToSiteSecure ? 3829 : 0); + } + + @Override + public String toString() { + return "Node[" + hostname + ":" + apiPort + "]"; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java new file mode 100644 index 0000000000..630631fcc1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/NodeInformationAdapter.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster; + +import javax.xml.bind.annotation.adapters.XmlAdapter; + +public class NodeInformationAdapter extends XmlAdapter { + + @Override + public NodeInformation unmarshal(final AdaptedNodeInformation adapted) throws Exception { + return new NodeInformation(adapted.getHostname(), adapted.getSiteToSitePort(), adapted.getApiPort(), adapted.isSiteToSiteSecure(), adapted.getTotalFlowFiles()); + } + + @Override + public AdaptedNodeInformation marshal(final NodeInformation nodeInformation) throws Exception { + final AdaptedNodeInformation adapted = new AdaptedNodeInformation(); + adapted.setHostname(nodeInformation.getHostname()); + adapted.setSiteToSitePort(nodeInformation.getSiteToSitePort()); + adapted.setApiPort(nodeInformation.getAPIPort()); + adapted.setSiteToSiteSecure(nodeInformation.isSiteToSiteSecure()); + adapted.setTotalFlowFiles(nodeInformation.getTotalFlowFiles()); + return adapted; + } + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java new file mode 100644 index 0000000000..57c1c30970 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/cluster/protocol/DataFlow.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.cluster.protocol; + +public interface DataFlow { + + /** + * @return the raw byte array of the flow + */ + public byte[] getFlow(); + + /** + * @return the raw byte array of the templates + */ + public byte[] getTemplates(); + + /** + * @return the raw byte array of the snippets + */ + public byte[] getSnippets(); + + /** + * @return true if processors should be automatically started at application + * startup; false otherwise + */ + public boolean isAutoStartProcessors(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java new file mode 100644 index 0000000000..3d5c75da7d --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connectable.java @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.connectable; + +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.Triggerable; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.scheduling.SchedulingStrategy; + +/** + * Represents a connectable component to which or from which data can flow. + */ +public interface Connectable extends Triggerable { + + /** + * @return the unique identifier for this Connectable + */ + String getIdentifier(); + + /** + * @return a Collection of all relationships for this Connectable + */ + Collection getRelationships(); + + /** + * Returns the ProcessorRelationship whose name is given + * + * @param relationshipName + * @return a ProcessorRelationship whose name is given, or null + * if none exists + */ + Relationship getRelationship(String relationshipName); + + /** + * Adds the given connection to this Connectable. + * + * @param connection the connection to add + * @throws NullPointerException if the argument is null + * @throws IllegalArgumentException if the given Connection is not valid + */ + void addConnection(Connection connection) throws IllegalArgumentException; + + /** + * @return true if the Connectable is the destination of any other + * Connectable, false otherwise. + */ + boolean hasIncomingConnection(); + + /** + * + * @param connection + * @throws IllegalStateException if the given Connection is not registered + * to this. + */ + void removeConnection(Connection connection) throws IllegalStateException; + + /** + * Updates any internal state that depends on the given connection. The + * given connection will share the same ID as the old connection. + * + * @param newConnection + * @throws IllegalStateException + */ + void updateConnection(Connection newConnection) throws IllegalStateException; + + /** + * @return a Set of all Connections for which this + * Connectable is the destination + */ + List getIncomingConnections(); + + /** + * @return a Set of all Connections for which this + * Connectable is the source; if no connections exist, will + * return an empty Collection. Guaranteed not null. + */ + Set getConnections(); + + /** + * @param relationship + * @return a Set of all Connections that contain + * the given relationship for which this Connectable is the + * source + */ + Set getConnections(Relationship relationship); + + /** + * Returns the position on the graph where this Connectable is located + * + * @return + */ + Position getPosition(); + + /** + * Updates this component's position on the graph + * + * @param position + */ + void setPosition(Position position); + + /** + * @return the name of this Connectable + */ + String getName(); + + /** + * Sets the name of this Connectable so that its name will be visible on the + * UI + * @param name + */ + void setName(String name); + + /** + * @return the comments of this Connectable + */ + String getComments(); + + /** + * Sets the comments of this Connectable. + * @param comments + */ + void setComments(String comments); + + /** + * If true, + * {@link #onTrigger(nifi.processor.ProcessContext, nifi.processor.ProcessSessionFactory)} + * should be called even when this Connectable has no FlowFiles queued for + * processing + * + * @return + */ + boolean isTriggerWhenEmpty(); + + /** + * Returns the ProcessGroup to which this Connectable belongs + * + * @return + */ + ProcessGroup getProcessGroup(); + + /** + * Sets the new ProcessGroup to which this Connectable belongs + * + * @param group + */ + void setProcessGroup(ProcessGroup group); + + /** + * + * @param relationship the relationship + * @return true indicates flow files transferred to the given relationship + * should be terminated if the relationship is not connected to another + * FlowFileConsumer; false indicates they will not be terminated and the + * processor will not be valid until specified + */ + boolean isAutoTerminated(Relationship relationship); + + /** + * Indicates whether flow file content made by this connectable must be + * persisted + * + * @return + */ + boolean isLossTolerant(); + + /** + * @param lossTolerant + */ + void setLossTolerant(boolean lossTolerant); + + /** + * @return the type of the Connectable + */ + ConnectableType getConnectableType(); + + /** + * Returns the any validation errors for this connectable. + * + * @return + */ + Collection getValidationErrors(); + + /** + * Returns the amount of time for which a FlowFile should be penalized when + * {@link ProcessSession#penalize(nifi.flowfile.FlowFile)} is called + * + * @param timeUnit + * @return + */ + long getPenalizationPeriod(final TimeUnit timeUnit); + + /** + * Returns a string representation for which a FlowFile should be penalized + * when {@link ProcessSession#penalize(nifi.flowfile.FlowFile)} is called + * + * @return + */ + String getPenalizationPeriod(); + + /** + * @param timeUnit determines the unit of time to represent the yield + * period. + * @return + */ + long getYieldPeriod(TimeUnit timeUnit); + + /** + * returns the string representation for this Connectable's configured yield + * period + * + * @return + */ + String getYieldPeriod(); + + /** + * Updates the amount of time that this Connectable should avoid being + * scheduled when the processor calls + * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()} + * + * @param yieldPeriod + */ + void setYieldPeriod(String yieldPeriod); + + /** + * Updates the amount of time that this Connectable will penalize FlowFiles + * when {@link ProcessSession#penalize(nifi.flowfile.FlowFile)} is called + * @param penalizationPeriod + */ + void setPenalizationPeriod(String penalizationPeriod); + + /** + * Causes the processor not to be scheduled for some period of time. This + * duration can be obtained and set via the + * {@link #getYieldPeriod(TimeUnit)} and + * {@link #setYieldPeriod(long, TimeUnit)} methods. + */ + void yield(); + + /** + * Returns the time in milliseconds since Epoch at which this Connectable + * should no longer yield its threads + * + * @return + */ + long getYieldExpiration(); + + /** + * Specifies whether or not this component is considered side-effect free, + * with respect to external systems. + * + * @return + */ + boolean isSideEffectFree(); + + void verifyCanDelete() throws IllegalStateException; + + void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException; + + void verifyCanStart() throws IllegalStateException; + + void verifyCanStop() throws IllegalStateException; + + void verifyCanUpdate() throws IllegalStateException; + + void verifyCanEnable() throws IllegalStateException; + + void verifyCanDisable() throws IllegalStateException; + + SchedulingStrategy getSchedulingStrategy(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java new file mode 100644 index 0000000000..0334bfbaa3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/ConnectableType.java @@ -0,0 +1,44 @@ +/* + * 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.connectable; + +import javax.xml.bind.annotation.XmlEnum; + +@XmlEnum +public enum ConnectableType { + + PROCESSOR, + /** + * Port that lives within an RemoteProcessGroup and is used to send data to + * remote NiFi instances + */ + REMOTE_INPUT_PORT, + /** + * Port that lives within a RemoteProcessGroup and is used to receive data + * from remote NiFi instances + */ + REMOTE_OUTPUT_PORT, + /** + * Root Group Input Ports and Local Input Ports + */ + INPUT_PORT, + /** + * Root Group Output Ports and Local Output Ports + */ + OUTPUT_PORT, + FUNNEL +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java new file mode 100644 index 0000000000..0a0089d966 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Connection.java @@ -0,0 +1,78 @@ +/* + * 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.connectable; + +import java.util.Collection; +import java.util.List; +import java.util.Set; + +import org.apache.nifi.controller.FlowFileQueue; +import org.apache.nifi.controller.repository.FlowFileRecord; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.processor.FlowFileFilter; +import org.apache.nifi.processor.Relationship; + +public interface Connection { + + void enqueue(FlowFileRecord flowFile); + + void enqueue(Collection flowFiles); + + Connectable getDestination(); + + Collection getRelationships(); + + FlowFileQueue getFlowFileQueue(); + + String getIdentifier(); + + String getName(); + + void setName(String name); + + void setBendPoints(List position); + + List getBendPoints(); + + int getLabelIndex(); + + void setLabelIndex(int labelIndex); + + long getZIndex(); + + void setZIndex(long zIndex); + + Connectable getSource(); + + void setRelationships(Collection newRelationships); + + void setDestination(final Connectable newDestination); + + void setProcessGroup(ProcessGroup processGroup); + + ProcessGroup getProcessGroup(); + + void lock(); + + void unlock(); + + List poll(FlowFileFilter filter, Set expiredRecords); + + void verifyCanUpdate() throws IllegalStateException; + + void verifyCanDelete() throws IllegalStateException; +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java new file mode 100644 index 0000000000..cceca8f225 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Funnel.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.connectable; + +import org.apache.nifi.controller.ScheduledState; + +public interface Funnel extends Connectable { + + void setScheduledState(ScheduledState scheduledState); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java new file mode 100644 index 0000000000..907dd9205f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Port.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.connectable; + +public interface Port extends Connectable { + + void shutdown(); + + boolean isValid(); + + /** + *

+ * This method is called just before a Port is scheduled to run, giving the + * Port a chance to initialize any resources needed.

+ */ + void onSchedulingStart(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.java new file mode 100644 index 0000000000..75d04f59c7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Position.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.connectable; + +public class Position { + + private final double x; + private final double y; + + public Position(final double x, final double y) { + this.x = x; + this.y = y; + } + + public double getX() { + return x; + } + + public double getY() { + return y; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.java new file mode 100644 index 0000000000..cea13d2e55 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/connectable/Size.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.connectable; + +public class Size { + + private final double width; + private final double height; + + public Size(final double width, final double height) { + this.width = width; + this.height = height; + } + + public double getWidth() { + return width; + } + + public double getHeight() { + return height; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java new file mode 100644 index 0000000000..ef4b72aad8 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractConfiguredComponent.java @@ -0,0 +1,280 @@ +/* + * 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.controller; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.nifi.components.ConfigurableComponent; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.service.ControllerServiceNode; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.nar.NarCloseable; + +public abstract class AbstractConfiguredComponent implements ConfigurableComponent, ConfiguredComponent { + + private final String id; + private final ConfigurableComponent component; + private final ValidationContextFactory validationContextFactory; + private final ControllerServiceProvider serviceProvider; + + private final AtomicReference name = new AtomicReference<>(); + private final AtomicReference annotationData = new AtomicReference<>(); + + private final Lock lock = new ReentrantLock(); + private final ConcurrentMap properties = new ConcurrentHashMap<>(); + + public AbstractConfiguredComponent(final ConfigurableComponent component, final String id, + final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) { + this.id = id; + this.component = component; + this.validationContextFactory = validationContextFactory; + this.serviceProvider = serviceProvider; + } + + @Override + public String getIdentifier() { + return id; + } + + @Override + public String getName() { + return name.get(); + } + + @Override + public void setName(final String name) { + this.name.set(Objects.requireNonNull(name).intern()); + } + + @Override + public String getAnnotationData() { + return annotationData.get(); + } + + @Override + public void setAnnotationData(final String data) { + annotationData.set(data); + } + + @Override + public void setProperty(final String name, final String value) { + if (null == name || null == value) { + throw new IllegalArgumentException(); + } + + lock.lock(); + try { + verifyModifiable(); + + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + final PropertyDescriptor descriptor = component.getPropertyDescriptor(name); + + final String oldValue = properties.put(descriptor, value); + if (!value.equals(oldValue)) { + + if (descriptor.getControllerServiceDefinition() != null) { + if (oldValue != null) { + final ControllerServiceNode oldNode = serviceProvider.getControllerServiceNode(oldValue); + if (oldNode != null) { + oldNode.removeReference(this); + } + } + + final ControllerServiceNode newNode = serviceProvider.getControllerServiceNode(value); + if (newNode != null) { + newNode.addReference(this); + } + } + + try { + component.onPropertyModified(descriptor, oldValue, value); + } catch (final Throwable t) { + // nothing really to do here... + } + } + } + } finally { + lock.unlock(); + } + } + + /** + * Removes the property and value for the given property name if a + * descriptor and value exists for the given name. If the property is + * optional its value might be reset to default or will be removed entirely + * if was a dynamic property. + * + * @param name the property to remove + * @return true if removed; false otherwise + * @throws java.lang.IllegalArgumentException if the name is null + */ + @Override + public boolean removeProperty(final String name) { + if (null == name) { + throw new IllegalArgumentException(); + } + + lock.lock(); + try { + verifyModifiable(); + + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + final PropertyDescriptor descriptor = component.getPropertyDescriptor(name); + String value = null; + if (!descriptor.isRequired() && (value = properties.remove(descriptor)) != null) { + component.onPropertyModified(descriptor, value, null); + return true; + } + } + } finally { + lock.unlock(); + } + return false; + } + + @Override + public Map getProperties() { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + final List supported = component.getPropertyDescriptors(); + if (supported == null || supported.isEmpty()) { + return Collections.unmodifiableMap(properties); + } else { + final Map props = new LinkedHashMap<>(); + for (final PropertyDescriptor descriptor : supported) { + props.put(descriptor, null); + } + props.putAll(properties); + return props; + } + } + } + + @Override + public String getProperty(final PropertyDescriptor property) { + return properties.get(property); + } + + @Override + public int hashCode() { + return 273171 * id.hashCode(); + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj == null) { + return false; + } + + if (!(obj instanceof ConfiguredComponent)) { + return false; + } + + final ConfiguredComponent other = (ConfiguredComponent) obj; + return id.equals(other.getIdentifier()); + } + + @Override + public String toString() { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return component.toString(); + } + } + + @Override + public Collection validate(final ValidationContext context) { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return component.validate(context); + } + } + + @Override + public PropertyDescriptor getPropertyDescriptor(final String name) { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return component.getPropertyDescriptor(name); + } + } + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + component.onPropertyModified(descriptor, oldValue, newValue); + } + } + + @Override + public List getPropertyDescriptors() { + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + return component.getPropertyDescriptors(); + } + } + + @Override + public boolean isValid() { + final Collection validationResults = validate(validationContextFactory.newValidationContext(getProperties(), getAnnotationData())); + for (final ValidationResult result : validationResults) { + if (!result.isValid()) { + return false; + } + } + + return true; + } + + @Override + public Collection getValidationErrors() { + final List results = new ArrayList<>(); + lock.lock(); + try { + final ValidationContext validationContext = validationContextFactory.newValidationContext(getProperties(), getAnnotationData()); + + final Collection validationResults; + try (final NarCloseable narCloseable = NarCloseable.withNarLoader()) { + validationResults = component.validate(validationContext); + } + + for (final ValidationResult result : validationResults) { + if (!result.isValid()) { + results.add(result); + } + } + } catch (final Throwable t) { + results.add(new ValidationResult.Builder().explanation("Failed to run validation due to " + t.toString()).valid(false).build()); + } finally { + lock.unlock(); + } + return results; + } + + public abstract void verifyModifiable() throws IllegalStateException; + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java new file mode 100644 index 0000000000..e1d2dd4e65 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/AbstractPort.java @@ -0,0 +1,636 @@ +/* + * 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.controller; + +import static java.util.Objects.requireNonNull; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableType; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.Port; +import org.apache.nifi.connectable.Position; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.util.FormatUtils; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +public abstract class AbstractPort implements Port { + + public static final Relationship PORT_RELATIONSHIP = new Relationship.Builder() + .description("The relationship through which all Flow Files are transferred") + .name("") + .build(); + + public static final long MINIMUM_PENALIZATION_MILLIS = 0L; + public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS; + + public static final long MINIMUM_YIELD_MILLIS = 0L; + public static final long DEFAULT_YIELD_PERIOD = 10000L; + public static final TimeUnit DEFAULT_YIELD_TIME_UNIT = TimeUnit.MILLISECONDS; + + private final List relationships; + + private final String id; + private final ConnectableType type; + private final AtomicReference name; + private final AtomicReference position; + private final AtomicReference comments; + private final AtomicReference processGroup; + private final AtomicBoolean lossTolerant; + private final AtomicReference scheduledState; + private final AtomicInteger concurrentTaskCount; + private final AtomicReference penalizationPeriod; + private final AtomicReference yieldPeriod; + private final AtomicReference schedulingPeriod; + private final AtomicLong schedulingNanos; + private final AtomicLong yieldExpiration; + private final ProcessScheduler processScheduler; + + private final Set outgoingConnections; + private final List incomingConnections; + + private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + private final Lock writeLock = rwLock.writeLock(); + + public AbstractPort(final String id, final String name, final ProcessGroup processGroup, final ConnectableType type, final ProcessScheduler scheduler) { + this.id = requireNonNull(id); + this.name = new AtomicReference<>(requireNonNull(name)); + position = new AtomicReference<>(new Position(0D, 0D)); + outgoingConnections = new HashSet<>(); + incomingConnections = new ArrayList<>(); + comments = new AtomicReference<>(); + lossTolerant = new AtomicBoolean(false); + concurrentTaskCount = new AtomicInteger(1); + processScheduler = scheduler; + + final List relationshipList = new ArrayList<>(); + relationshipList.add(PORT_RELATIONSHIP); + relationships = Collections.unmodifiableList(relationshipList); + this.processGroup = new AtomicReference<>(processGroup); + this.type = type; + penalizationPeriod = new AtomicReference<>("30 sec"); + yieldPeriod = new AtomicReference<>("1 sec"); + yieldExpiration = new AtomicLong(0L); + schedulingPeriod = new AtomicReference<>("0 millis"); + schedulingNanos = new AtomicLong(30000); + scheduledState = new AtomicReference<>(ScheduledState.STOPPED); + } + + @Override + public String getIdentifier() { + return id; + } + + @Override + public String getName() { + return name.get(); + } + + @Override + public void setName(final String name) { + if (this.name.get().equals(name)) { + return; + } + + final ProcessGroup parentGroup = this.processGroup.get(); + if (getConnectableType() == ConnectableType.INPUT_PORT) { + if (parentGroup.getInputPortByName(name) != null) { + throw new IllegalStateException("Cannot rename port from " + this.name.get() + " to " + name + " because the ProcessGroup already has an Input Port named " + name); + } + } else if (getConnectableType() == ConnectableType.OUTPUT_PORT) { + if (parentGroup.getOutputPortByName(name) != null) { + throw new IllegalStateException("Cannot rename port from " + this.name.get() + " to " + name + " because the ProcessGroup already has an Output Port named " + name); + } + } + + this.name.set(name); + } + + @Override + public ProcessGroup getProcessGroup() { + return processGroup.get(); + } + + @Override + public void setProcessGroup(final ProcessGroup newGroup) { + this.processGroup.set(newGroup); + } + + @Override + public String getComments() { + return comments.get(); + } + + @Override + public void setComments(final String comments) { + this.comments.set(comments); + } + + @Override + public Collection getRelationships() { + return relationships; + } + + @Override + public Relationship getRelationship(final String relationshipName) { + if (PORT_RELATIONSHIP.getName().equals(relationshipName)) { + return PORT_RELATIONSHIP; + } + return null; + } + + @Override + public void addConnection(final Connection connection) throws IllegalArgumentException { + writeLock.lock(); + try { + if (!requireNonNull(connection).getSource().equals(this)) { + if (connection.getDestination().equals(this)) { + // don't add the connection twice. This may occur if we have a self-loop because we will be told + // to add the connection once because we are the source and again because we are the destination. + if (!incomingConnections.contains(connection)) { + incomingConnections.add(connection); + } + + return; + } else { + throw new IllegalArgumentException("Cannot add a connection to a LocalPort for which the LocalPort is neither the Source nor the Destination"); + } + } + + for (final Relationship relationship : connection.getRelationships()) { + if (!relationship.equals(PORT_RELATIONSHIP)) { + throw new IllegalArgumentException("No relationship with name " + relationship + " exists for Local Ports"); + } + } + + // don't add the connection twice. This may occur if we have a self-loop because we will be told + // to add the connection once because we are the source and again because we are the destination. + if (!outgoingConnections.contains(connection)) { + outgoingConnections.add(connection); + } + } finally { + writeLock.unlock(); + } + } + + @Override + public boolean hasIncomingConnection() { + readLock.lock(); + try { + return !incomingConnections.isEmpty(); + } finally { + readLock.unlock(); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException { + final ProcessSession session = sessionFactory.createSession(); + + try { + onTrigger(context, session); + session.commit(); + } catch (final ProcessException e) { + session.rollback(); + throw e; + } catch (final Throwable t) { + session.rollback(); + throw new RuntimeException(t); + } + } + + public abstract void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException; + + @Override + public void updateConnection(final Connection connection) throws IllegalStateException { + if (requireNonNull(connection).getSource().equals(this)) { + writeLock.lock(); + try { + if (!outgoingConnections.remove(connection)) { + throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port"); + } + outgoingConnections.add(connection); + } finally { + writeLock.unlock(); + } + } else if (connection.getDestination().equals(this)) { + writeLock.lock(); + try { + if (!incomingConnections.remove(connection)) { + throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port"); + } + incomingConnections.add(connection); + } finally { + writeLock.unlock(); + } + } else { + throw new IllegalStateException("The given connection is not currently registered for this Port"); + } + } + + @Override + public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException { + writeLock.lock(); + try { + if (!requireNonNull(connection).getSource().equals(this)) { + final boolean existed = incomingConnections.remove(connection); + if (!existed) { + throw new IllegalStateException("The given connection is not currently registered for this Port"); + } + return; + } + + if (!canConnectionBeRemoved(connection)) { + // TODO: Determine which processors will be broken if connection is removed, rather than just returning a boolean + throw new IllegalStateException(connection + " cannot be removed"); + } + + final boolean removed = outgoingConnections.remove(connection); + if (!removed) { + throw new IllegalStateException(connection + " is not registered with " + this); + } + } finally { + writeLock.unlock(); + } + } + + /** + * Verify that removing this connection will not prevent this Port from + * still being connected via each relationship + * + * @param connection + * @return + */ + private boolean canConnectionBeRemoved(final Connection connection) { + final Connectable source = connection.getSource(); + if (!source.isRunning()) { + // we don't have to verify that this Connectable is still connected because it's okay to make + // the source invalid since it is not running. + return true; + } + + for (final Relationship relationship : source.getRelationships()) { + if (source.isAutoTerminated(relationship)) { + continue; + } + + final Set connectionsForRelationship = source.getConnections(relationship); + if (connectionsForRelationship == null || connectionsForRelationship.isEmpty()) { + return false; + } + } + + return true; + } + + @Override + public Set getConnections() { + readLock.lock(); + try { + return Collections.unmodifiableSet(outgoingConnections); + } finally { + readLock.unlock(); + } + } + + @Override + public Set getConnections(final Relationship relationship) { + readLock.lock(); + try { + if (relationship.equals(PORT_RELATIONSHIP)) { + return Collections.unmodifiableSet(outgoingConnections); + } + + throw new IllegalArgumentException("No relationship with name " + relationship.getName() + " exists for Local Ports"); + } finally { + readLock.unlock(); + } + } + + @Override + public Position getPosition() { + return position.get(); + } + + @Override + public void setPosition(final Position position) { + this.position.set(position); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("name", getName()).append("id", getIdentifier()).toString(); + } + + @Override + public List getIncomingConnections() { + readLock.lock(); + try { + return Collections.unmodifiableList(incomingConnections); + } finally { + readLock.unlock(); + } + } + + /** + * Indicates whether or not this Port is valid. + * + * @return + */ + @Override + public abstract boolean isValid(); + + @Override + public boolean isAutoTerminated(final Relationship relationship) { + return false; + } + + @Override + public boolean isLossTolerant() { + return lossTolerant.get(); + } + + @Override + public void setLossTolerant(boolean lossTolerant) { + this.lossTolerant.set(lossTolerant); + } + + @Override + public void setMaxConcurrentTasks(final int taskCount) { + if (taskCount < 1) { + throw new IllegalArgumentException(); + } + concurrentTaskCount.set(taskCount); + } + + /** + * @return the number of tasks that may execute concurrently for this + * processor + */ + @Override + public int getMaxConcurrentTasks() { + return concurrentTaskCount.get(); + } + + /** + * + */ + @Override + public void shutdown() { + scheduledState.set(ScheduledState.STOPPED); + } + + @Override + public void onSchedulingStart() { + scheduledState.set(ScheduledState.RUNNING); + } + + public void disable() { + final boolean updated = scheduledState.compareAndSet(ScheduledState.STOPPED, ScheduledState.DISABLED); + if (!updated) { + throw new IllegalStateException("Port cannot be disabled because it is not stopped"); + } + } + + public void enable() { + final boolean updated = scheduledState.compareAndSet(ScheduledState.DISABLED, ScheduledState.STOPPED); + if (!updated) { + throw new IllegalStateException("Port cannot be enabled because it is not disabled"); + } + } + + @Override + public boolean isRunning() { + return getScheduledState().equals(ScheduledState.RUNNING) || processScheduler.getActiveThreadCount(this) > 0; + } + + @Override + public ScheduledState getScheduledState() { + return scheduledState.get(); + } + + @Override + public ConnectableType getConnectableType() { + return type; + } + + /** + * Updates the amount of time that this processor should avoid being + * scheduled when the processor calls + * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()} + * + * @param yieldPeriod + */ + @Override + public void setYieldPeriod(final String yieldPeriod) { + final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS); + if (yieldMillis < 0) { + throw new IllegalArgumentException("Yield duration must be positive"); + } + this.yieldPeriod.set(yieldPeriod); + } + + /** + * @param schedulingPeriod + */ + @Override + public void setScheduldingPeriod(final String schedulingPeriod) { + final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); + if (schedulingNanos < 0) { + throw new IllegalArgumentException("Scheduling Period must be positive"); + } + + this.schedulingPeriod.set(schedulingPeriod); + this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos)); + } + + @Override + public long getPenalizationPeriod(final TimeUnit timeUnit) { + return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit); + } + + @Override + public String getPenalizationPeriod() { + return penalizationPeriod.get(); + } + + /** + * Causes the processor not to be scheduled for some period of time. This + * duration can be obtained and set via the + * {@link #getYieldPeriod(TimeUnit)} and + * {@link #setYieldPeriod(long, TimeUnit)} methods. + */ + @Override + public void yield() { + final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS); + yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis)); + } + + @Override + public long getYieldExpiration() { + return yieldExpiration.get(); + } + + @Override + public long getSchedulingPeriod(final TimeUnit timeUnit) { + return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS); + } + + @Override + public String getSchedulingPeriod() { + return schedulingPeriod.get(); + } + + @Override + public void setPenalizationPeriod(final String penalizationPeriod) { + this.penalizationPeriod.set(penalizationPeriod); + } + + @Override + public String getYieldPeriod() { + return yieldPeriod.get(); + } + + @Override + public long getYieldPeriod(final TimeUnit timeUnit) { + return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit); + } + + @Override + public void verifyCanDelete() throws IllegalStateException { + verifyCanDelete(false); + } + + @Override + public void verifyCanDelete(final boolean ignoreConnections) { + readLock.lock(); + try { + if (isRunning()) { + throw new IllegalStateException(this + " is running"); + } + + if (!ignoreConnections) { + for (final Connection connection : outgoingConnections) { + connection.verifyCanDelete(); + } + + for (final Connection connection : incomingConnections) { + if (connection.getSource().equals(this)) { + connection.verifyCanDelete(); + } else { + throw new IllegalStateException(this + " is the destination of another component"); + } + } + } + } finally { + readLock.unlock(); + } + } + + @Override + public void verifyCanStart() { + readLock.lock(); + try { + if (scheduledState.get() != ScheduledState.STOPPED) { + throw new IllegalStateException(this + " is not stopped"); + } + verifyNoActiveThreads(); + + final Collection validationResults = getValidationErrors(); + if (!validationResults.isEmpty()) { + throw new IllegalStateException(this + " is not in a valid state: " + validationResults.iterator().next().getExplanation()); + } + } finally { + readLock.unlock(); + } + } + + @Override + public void verifyCanStop() { + if (getScheduledState() != ScheduledState.RUNNING) { + throw new IllegalStateException(this + " is not scheduled to run"); + } + } + + @Override + public void verifyCanUpdate() { + readLock.lock(); + try { + if (isRunning()) { + throw new IllegalStateException(this + " is not stopped"); + } + } finally { + readLock.unlock(); + } + } + + @Override + public void verifyCanEnable() { + readLock.lock(); + try { + if (getScheduledState() != ScheduledState.DISABLED) { + throw new IllegalStateException(this + " is not disabled"); + } + + verifyNoActiveThreads(); + } finally { + readLock.unlock(); + } + } + + @Override + public void verifyCanDisable() { + readLock.lock(); + try { + if (getScheduledState() != ScheduledState.STOPPED) { + throw new IllegalStateException(this + " is not stopped"); + } + verifyNoActiveThreads(); + } finally { + readLock.unlock(); + } + } + + private void verifyNoActiveThreads() throws IllegalStateException { + final int threadCount = processScheduler.getActiveThreadCount(this); + if (threadCount > 0) { + throw new IllegalStateException(this + " has " + threadCount + " threads still active"); + } + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java new file mode 100644 index 0000000000..38df6f754e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Availability.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller; + +public enum Availability { + + CLUSTER_MANAGER_ONLY, + NODE_ONLY, + BOTH; +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java new file mode 100644 index 0000000000..5b95524bc6 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ConfiguredComponent.java @@ -0,0 +1,63 @@ +/* + * 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.controller; + +import java.util.Collection; +import java.util.Map; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationResult; + +public interface ConfiguredComponent { + + public String getIdentifier(); + + public String getName(); + + public void setName(String name); + + public String getAnnotationData(); + + public void setAnnotationData(String data); + + public void setProperty(String name, String value); + + /** + * Removes the property and value for the given property name if a + * descriptor and value exists for the given name. If the property is + * optional its value might be reset to default or will be removed entirely + * if was a dynamic property. + * + * @param name the property to remove + * @return true if removed; false otherwise + * @throws java.lang.IllegalArgumentException if the name is null + */ + public boolean removeProperty(String name); + + public Map getProperties(); + + public String getProperty(final PropertyDescriptor property); + + boolean isValid(); + + /** + * Returns the any validation errors for this connectable. + * + * @return + */ + Collection getValidationErrors(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java new file mode 100644 index 0000000000..eee878e04c --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ContentAvailability.java @@ -0,0 +1,65 @@ +/* + * 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.controller; + +/** + * Provides information about whether or not the data referenced in a Provenance + * Event can be replayed or downloaded + */ +public interface ContentAvailability { + + /** + * Returns a boolean indicating whether or not the Input content is + * available + * + * @return + */ + boolean isInputAvailable(); + + /** + * Returns a boolean indicating whether or not the Output content is + * available + * + * @return + */ + boolean isOutputAvailable(); + + /** + * Returns true if the Input content is the same as the Output + * content + * + * @return + */ + boolean isContentSame(); + + /** + * Returns a boolean indicating whether or not the content is replayable. If + * this returns false, the reason that replay is not available + * can be determined by calling {@link #getReasonNotReplayable()}. + * + * @return + */ + boolean isReplayable(); + + /** + * Returns the reason that the content cannot be replayed, or + * null if the content can be replayed. + * + * @return + */ + String getReasonNotReplayable(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java new file mode 100644 index 0000000000..eaa0c48fc1 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Counter.java @@ -0,0 +1,32 @@ +/* + * 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.controller; + +public interface Counter { + + void adjust(long delta); + + String getName(); + + long getValue(); + + String getContext(); + + String getIdentifier(); + + void reset(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java new file mode 100644 index 0000000000..280f69df73 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/EventBasedWorker.java @@ -0,0 +1,32 @@ +/* + * 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.controller; + +import org.apache.nifi.connectable.Connectable; + +/** + * Wraps a Connectable object and maintains a count of how many unanswered + * events have been reported for the Connectable + */ +public interface EventBasedWorker { + + Connectable getConnectable(); + + int incrementEventCount(); + + int decrementEventCount(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java new file mode 100644 index 0000000000..1195bc97a4 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/Heartbeater.java @@ -0,0 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller; + +public interface Heartbeater { + + void heartbeat(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java new file mode 100644 index 0000000000..303f5401ad --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessScheduler.java @@ -0,0 +1,146 @@ +/* + * 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.controller; + +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.Funnel; +import org.apache.nifi.connectable.Port; +import org.apache.nifi.processor.annotation.OnScheduled; +import org.apache.nifi.processor.annotation.OnUnscheduled; +import org.apache.nifi.scheduling.SchedulingStrategy; + +public interface ProcessScheduler { + + /** + * Shuts down the scheduler, stopping all components + */ + void shutdown(); + + /** + * Starts scheduling the given processor to run after invoking all methods + * on the underlying {@link nifi.processor.Processor FlowFileProcessor} that + * are annotated with the {@link OnScheduled} annotation. If the Processor + * is already scheduled to run, does nothing. + * + * @param procNode + * @throws IllegalStateException if the Processor is disabled + */ + void startProcessor(ProcessorNode procNode); + + /** + * Stops scheduling the given processor to run and invokes all methods on + * the underlying {@link nifi.processor.Processor FlowFileProcessor} that + * are annotated with the {@link OnUnscheduled} annotation. This does not + * interrupt any threads that are currently running within the given + * Processor. If the Processor is not scheduled to run, does nothing. + * @param procNode + */ + void stopProcessor(ProcessorNode procNode); + + /** + * Starts scheduling the given Port to run. If the Port is already scheduled + * to run, does nothing. + * + * @param port + * + * @throws IllegalStateException if the Port is disabled + */ + void startPort(Port port); + + /** + * Stops scheduling the given Port to run. This does not interrupt any + * threads that are currently running within the given port. This does not + * interrupt any threads that are currently running within the given Port. + * If the Port is not scheduled to run, does nothing. + * + * @param port + */ + void stopPort(Port port); + + /** + * Starts scheduling the given Funnel to run. If the funnel is already + * scheduled to run, does nothing. + * + * @param funnel + * + * @throws IllegalStateException if the Funnel is disabled + */ + void startFunnel(Funnel funnel); + + /** + * Stops scheduling the given Funnel to run. This does not interrupt any + * threads that are currently running within the given funnel. If the funnel + * is not scheduled to run, does nothing. + * + * @param funnel + */ + void stopFunnel(Funnel funnel); + + void enableFunnel(Funnel funnel); + + void enablePort(Port port); + + void enableProcessor(ProcessorNode procNode); + + void disableFunnel(Funnel funnel); + + void disablePort(Port port); + + void disableProcessor(ProcessorNode procNode); + + /** + * Returns the number of threads currently active for the given + * Connectable. + * + * @param scheduled + * @return + */ + int getActiveThreadCount(Object scheduled); + + /** + * Returns a boolean indicating whether or not the given object is scheduled + * to run + * + * @param scheduled + * @return + */ + boolean isScheduled(Object scheduled); + + /** + * Registers a relevant event for an Event-Driven worker + * + * @param worker + */ + void registerEvent(Connectable worker); + + /** + * Notifies the ProcessScheduler of how many threads are available to use + * for the given {@link SchedulingStrategy} + * + * @param strategy + * @param maxThreadCount + */ + void setMaxThreadCount(SchedulingStrategy strategy, int maxThreadCount); + + /** + * Notifies the Scheduler that it should stop scheduling the given component + * until its yield duration has expired + * + * @param procNode + */ + void yield(ProcessorNode procNode); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java new file mode 100644 index 0000000000..f6786fa9a5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ProcessorNode.java @@ -0,0 +1,80 @@ +/* + * 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.controller; + +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.controller.service.ControllerServiceProvider; +import org.apache.nifi.logging.LogLevel; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.scheduling.SchedulingStrategy; + +public abstract class ProcessorNode extends AbstractConfiguredComponent implements Connectable { + + public ProcessorNode(final Processor processor, final String id, + final ValidationContextFactory validationContextFactory, final ControllerServiceProvider serviceProvider) { + super(processor, id, validationContextFactory, serviceProvider); + } + + public abstract boolean isIsolated(); + + public abstract boolean isTriggerWhenAnyDestinationAvailable(); + + @Override + public abstract boolean isSideEffectFree(); + + public abstract boolean isTriggeredSerially(); + + public abstract boolean isEventDrivenSupported(); + + public abstract boolean isHighThroughputSupported(); + + @Override + public abstract boolean isValid(); + + public abstract void setScheduledState(ScheduledState scheduledState); + + public abstract void setBulletinLevel(LogLevel bulletinLevel); + + public abstract LogLevel getBulletinLevel(); + + public abstract Processor getProcessor(); + + public abstract void yield(long period, TimeUnit timeUnit); + + public abstract void setAutoTerminatedRelationships(Set relationships); + + public abstract Set getAutoTerminatedRelationships(); + + public abstract void setSchedulingStrategy(SchedulingStrategy schedulingStrategy); + + @Override + public abstract SchedulingStrategy getSchedulingStrategy(); + + public abstract void setRunDuration(long duration, TimeUnit timeUnit); + + public abstract long getRunDuration(TimeUnit timeUnit); + + public abstract Map getStyle(); + + public abstract void setStyle(Map style); + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java new file mode 100644 index 0000000000..6b8ede02c7 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ReportingTaskNode.java @@ -0,0 +1,56 @@ +/* + * 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.controller; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.reporting.ReportingContext; +import org.apache.nifi.reporting.ReportingTask; +import org.apache.nifi.scheduling.SchedulingStrategy; + +public interface ReportingTaskNode extends ConfiguredComponent { + + Availability getAvailability(); + + void setAvailability(Availability availability); + + void setSchedulingStrategy(SchedulingStrategy schedulingStrategy); + + SchedulingStrategy getSchedulingStrategy(); + + /** + * @return a string representation of the time between each scheduling + * period + */ + String getSchedulingPeriod(); + + long getSchedulingPeriod(TimeUnit timeUnit); + + /** + * Updates how often the ReportingTask should be triggered to run + * @param schedulingPeriod + */ + void setScheduldingPeriod(String schedulingPeriod); + + ReportingTask getReportingTask(); + + ReportingContext getReportingContext(); + + ConfigurationContext getConfigurationContext(); + + boolean isRunning(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java new file mode 100644 index 0000000000..3b880bb50f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFlowFileQueue.java @@ -0,0 +1,1093 @@ +/* + * 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.controller; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.nifi.connectable.Connection; +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.FlowFilePrioritizer; +import org.apache.nifi.processor.DataUnit; +import org.apache.nifi.processor.FlowFileFilter; +import org.apache.nifi.processor.FlowFileFilter.FlowFileFilterResult; +import org.apache.nifi.processor.QueueSize; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.FormatUtils; +import org.apache.nifi.util.concurrency.TimedLock; +import org.apache.nifi.util.timebuffer.LongEntityAccess; +import org.apache.nifi.util.timebuffer.TimedBuffer; +import org.apache.nifi.util.timebuffer.TimestampedLong; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A FlowFileQueue is used to queue FlowFile objects that are awaiting further + * processing. Must be thread safe. + * + * @author none + */ +public final class StandardFlowFileQueue implements FlowFileQueue { + + public static final int MAX_EXPIRED_RECORDS_PER_ITERATION = 100000; + public static final int SWAP_RECORD_POLL_SIZE = 10000; + + // When we have very high contention on a FlowFile Queue, the writeLock quickly becomes the bottleneck. In order to avoid this, + // we keep track of how often we are obtaining the write lock. If we exceed some threshold, we start performing a Pre-fetch so that + // we can then poll many times without having to obtain the lock. + // If lock obtained an average of more than PREFETCH_POLL_THRESHOLD times per second in order to poll from queue for last 5 seconds, do a pre-fetch. + public static final int PREFETCH_POLL_THRESHOLD = 1000; + public static final int PRIORITIZED_PREFETCH_SIZE = 10; + public static final int UNPRIORITIZED_PREFETCH_SIZE = 1000; + private volatile int prefetchSize = UNPRIORITIZED_PREFETCH_SIZE; // when we pre-fetch, how many should we pre-fetch? + + private static final Logger logger = LoggerFactory.getLogger(StandardFlowFileQueue.class); + + private PriorityQueue activeQueue = null; + private long activeQueueContentSize = 0L; + private ArrayList swapQueue = null; + + private int swappedRecordCount = 0; + private long swappedContentSize = 0L; + private String maximumQueueDataSize; + private long maximumQueueByteCount; + private boolean swapMode = false; + private long maximumQueueObjectCount; + + private final AtomicLong flowFileExpirationMillis; + private final Connection connection; + private final AtomicReference flowFileExpirationPeriod; + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); + private final List priorities; + private final int swapThreshold; + private final TimedLock readLock; + private final TimedLock writeLock; + private final String identifier; + + private final AtomicBoolean queueFullRef = new AtomicBoolean(false); + private final AtomicInteger activeQueueSizeRef = new AtomicInteger(0); + private final AtomicReference unacknowledgedSizeRef = new AtomicReference<>(new QueueSize(0, 0L)); + + // SCHEDULER CANNOT BE NOTIFIED OF EVENTS WITH THE WRITE LOCK HELD! DOING SO WILL RESULT IN A DEADLOCK! + private final ProcessScheduler scheduler; + + public StandardFlowFileQueue(final String identifier, final Connection connection, final ProcessScheduler scheduler, final int swapThreshold) { + activeQueue = new PriorityQueue<>(20, new Prioritizer(new ArrayList())); + priorities = new ArrayList<>(); + maximumQueueObjectCount = 0L; + maximumQueueDataSize = "0 MB"; + maximumQueueByteCount = 0L; + flowFileExpirationMillis = new AtomicLong(0); + flowFileExpirationPeriod = new AtomicReference<>("0 mins"); + swapQueue = new ArrayList<>(); + + this.identifier = identifier; + this.swapThreshold = swapThreshold; + this.scheduler = scheduler; + this.connection = connection; + + readLock = new TimedLock(this.lock.readLock(), identifier + " Read Lock", 100); + writeLock = new TimedLock(this.lock.writeLock(), identifier + " Write Lock", 100); + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public List getPriorities() { + return Collections.unmodifiableList(priorities); + } + + @Override + public int getSwapThreshold() { + return swapThreshold; + } + + @Override + public void setPriorities(final List newPriorities) { + writeLock.lock(); + try { + final PriorityQueue newQueue = new PriorityQueue<>(Math.max(20, activeQueue.size()), new Prioritizer(newPriorities)); + newQueue.addAll(activeQueue); + activeQueue = newQueue; + priorities.clear(); + priorities.addAll(newPriorities); + + if (newPriorities.isEmpty()) { + prefetchSize = UNPRIORITIZED_PREFETCH_SIZE; + } else { + prefetchSize = PRIORITIZED_PREFETCH_SIZE; + } + } finally { + writeLock.unlock("setPriorities"); + } + } + + @Override + public void setBackPressureObjectThreshold(final long maxQueueSize) { + writeLock.lock(); + try { + maximumQueueObjectCount = maxQueueSize; + this.queueFullRef.set(determineIfFull()); + } finally { + writeLock.unlock("setBackPressureObjectThreshold"); + } + } + + @Override + public long getBackPressureObjectThreshold() { + readLock.lock(); + try { + return maximumQueueObjectCount; + } finally { + readLock.unlock("getBackPressureObjectThreshold"); + } + } + + @Override + public void setBackPressureDataSizeThreshold(final String maxDataSize) { + writeLock.lock(); + try { + maximumQueueByteCount = DataUnit.parseDataSize(maxDataSize, DataUnit.B).longValue(); + maximumQueueDataSize = maxDataSize; + this.queueFullRef.set(determineIfFull()); + } finally { + writeLock.unlock("setBackPressureDataSizeThreshold"); + } + } + + @Override + public String getBackPressureDataSizeThreshold() { + readLock.lock(); + try { + return maximumQueueDataSize; + } finally { + readLock.unlock("getBackPressureDataSizeThreshold"); + } + } + + @Override + public QueueSize size() { + readLock.lock(); + try { + return getQueueSize(); + } finally { + readLock.unlock("getSize"); + } + } + + /** + * MUST be called with lock held + * + * @return + */ + private QueueSize getQueueSize() { + final QueueSize unacknowledged = unacknowledgedSizeRef.get(); + final PreFetch preFetch = preFetchRef.get(); + + final int preFetchCount; + final long preFetchSize; + if (preFetch == null) { + preFetchCount = 0; + preFetchSize = 0L; + } else { + final QueueSize preFetchQueueSize = preFetch.size(); + preFetchCount = preFetchQueueSize.getObjectCount(); + preFetchSize = preFetchQueueSize.getByteCount(); + } + + return new QueueSize(activeQueue.size() + swappedRecordCount + unacknowledged.getObjectCount() + preFetchCount, + activeQueueContentSize + swappedContentSize + unacknowledged.getByteCount() + preFetchSize); + } + + @Override + public long contentSize() { + readLock.lock(); + try { + final PreFetch prefetch = preFetchRef.get(); + if (prefetch == null) { + return activeQueueContentSize + swappedContentSize + unacknowledgedSizeRef.get().getObjectCount(); + } else { + return activeQueueContentSize + swappedContentSize + unacknowledgedSizeRef.get().getObjectCount() + prefetch.size().getByteCount(); + } + } finally { + readLock.unlock("getContentSize"); + } + } + + @Override + public boolean isEmpty() { + readLock.lock(); + try { + final PreFetch prefetch = preFetchRef.get(); + if (prefetch == null) { + return activeQueue.isEmpty() && swappedRecordCount == 0 && unacknowledgedSizeRef.get().getObjectCount() == 0; + } else { + return activeQueue.isEmpty() && swappedRecordCount == 0 && unacknowledgedSizeRef.get().getObjectCount() == 0 && prefetch.size().getObjectCount() == 0; + } + } finally { + readLock.unlock("isEmpty"); + } + } + + @Override + public boolean isActiveQueueEmpty() { + final int activeQueueSize = activeQueueSizeRef.get(); + if (activeQueueSize == 0) { + final PreFetch preFetch = preFetchRef.get(); + if (preFetch == null) { + return true; + } + + final QueueSize queueSize = preFetch.size(); + return queueSize.getObjectCount() == 0; + } else { + return false; + } + } + + @Override + public QueueSize getActiveQueueSize() { + readLock.lock(); + try { + final PreFetch preFetch = preFetchRef.get(); + if (preFetch == null) { + return new QueueSize(activeQueue.size(), activeQueueContentSize); + } else { + final QueueSize preFetchSize = preFetch.size(); + return new QueueSize(activeQueue.size() + preFetchSize.getObjectCount(), activeQueueContentSize + preFetchSize.getByteCount()); + } + } finally { + readLock.unlock("getActiveQueueSize"); + } + } + + @Override + public void acknowledge(final FlowFileRecord flowFile) { + if (queueFullRef.get()) { + writeLock.lock(); + try { + updateUnacknowledgedSize(-1, -flowFile.getSize()); + queueFullRef.set(determineIfFull()); + } finally { + writeLock.unlock("acknowledge(FlowFileRecord)"); + } + } else { + updateUnacknowledgedSize(-1, -flowFile.getSize()); + } + + if (connection.getSource().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) { + // queue was full but no longer is. Notify that the source may now be available to run, + // because of back pressure caused by this queue. + scheduler.registerEvent(connection.getSource()); + } + } + + @Override + public void acknowledge(final Collection flowFiles) { + long totalSize = 0L; + for (final FlowFileRecord flowFile : flowFiles) { + totalSize += flowFile.getSize(); + } + + if (queueFullRef.get()) { + writeLock.lock(); + try { + updateUnacknowledgedSize(-flowFiles.size(), -totalSize); + queueFullRef.set(determineIfFull()); + } finally { + writeLock.unlock("acknowledge(FlowFileRecord)"); + } + } else { + updateUnacknowledgedSize(-flowFiles.size(), -totalSize); + } + + if (connection.getSource().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) { + // it's possible that queue was full but no longer is. Notify that the source may now be available to run, + // because of back pressure caused by this queue. + scheduler.registerEvent(connection.getSource()); + } + } + + @Override + public boolean isFull() { + return queueFullRef.get(); + } + + /** + * MUST be called with either the read or write lock held + * + * @return + */ + private boolean determineIfFull() { + final long maxSize = maximumQueueObjectCount; + final long maxBytes = maximumQueueByteCount; + if (maxSize <= 0 && maxBytes <= 0) { + return false; + } + + final QueueSize queueSize = getQueueSize(); + if (maxSize > 0 && queueSize.getObjectCount() >= maxSize) { + return true; + } + + if (maxBytes > 0 && (queueSize.getByteCount() >= maxBytes)) { + return true; + } + + return false; + } + + @Override + public void put(final FlowFileRecord file) { + writeLock.lock(); + try { + if (swapMode || activeQueue.size() >= swapThreshold) { + swapQueue.add(file); + swappedContentSize += file.getSize(); + swappedRecordCount++; + swapMode = true; + } else { + activeQueueContentSize += file.getSize(); + activeQueue.add(file); + } + + queueFullRef.set(determineIfFull()); + } finally { + activeQueueSizeRef.set(activeQueue.size()); + writeLock.unlock("put(FlowFileRecord)"); + } + + if (connection.getDestination().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) { + scheduler.registerEvent(connection.getDestination()); + } + } + + @Override + public void putAll(final Collection files) { + final int numFiles = files.size(); + long bytes = 0L; + for (final FlowFile flowFile : files) { + bytes += flowFile.getSize(); + } + + writeLock.lock(); + try { + if (swapMode || activeQueue.size() >= swapThreshold - numFiles) { + swapQueue.addAll(files); + swappedContentSize += bytes; + swappedRecordCount += numFiles; + swapMode = true; + } else { + activeQueueContentSize += bytes; + activeQueue.addAll(files); + } + + queueFullRef.set(determineIfFull()); + } finally { + activeQueueSizeRef.set(activeQueue.size()); + writeLock.unlock("putAll"); + } + + if (connection.getDestination().getSchedulingStrategy() == SchedulingStrategy.EVENT_DRIVEN) { + scheduler.registerEvent(connection.getDestination()); + } + } + + @Override + public List pollSwappableRecords() { + writeLock.lock(); + try { + if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) { + return null; + } + + final List swapRecords = new ArrayList<>(Math.min(SWAP_RECORD_POLL_SIZE, swapQueue.size())); + final Iterator itr = swapQueue.iterator(); + while (itr.hasNext() && swapRecords.size() < SWAP_RECORD_POLL_SIZE) { + FlowFileRecord record = itr.next(); + swapRecords.add(record); + itr.remove(); + } + + swapQueue.trimToSize(); + return swapRecords; + } finally { + writeLock.unlock("pollSwappableRecords"); + } + } + + @Override + public void putSwappedRecords(final Collection records) { + writeLock.lock(); + try { + try { + for (final FlowFileRecord record : records) { + swappedContentSize -= record.getSize(); + swappedRecordCount--; + activeQueueContentSize += record.getSize(); + activeQueue.add(record); + } + + if (swappedRecordCount > swapQueue.size()) { + // we have more swap files to be swapped in. + return; + } + + // If a call to #pollSwappableRecords will not produce any, go ahead and roll those FlowFiles back into the mix + if (swapQueue.size() < SWAP_RECORD_POLL_SIZE) { + for (final FlowFileRecord record : swapQueue) { + activeQueue.add(record); + activeQueueContentSize += record.getSize(); + } + swapQueue.clear(); + swappedContentSize = 0L; + swappedRecordCount = 0; + swapMode = false; + } + } finally { + activeQueueSizeRef.set(activeQueue.size()); + } + } finally { + writeLock.unlock("putSwappedRecords"); + scheduler.registerEvent(connection.getDestination()); + } + } + + @Override + public void incrementSwapCount(final int numRecords, final long contentSize) { + writeLock.lock(); + try { + swappedContentSize += contentSize; + swappedRecordCount += numRecords; + } finally { + writeLock.unlock("incrementSwapCount"); + } + } + + @Override + public int unswappedSize() { + readLock.lock(); + try { + return activeQueue.size() + unacknowledgedSizeRef.get().getObjectCount(); + } finally { + readLock.unlock("unswappedSize"); + } + } + + @Override + public int getSwapRecordCount() { + readLock.lock(); + try { + return swappedRecordCount; + } finally { + readLock.unlock("getSwapRecordCount"); + } + } + + @Override + public int getSwapQueueSize() { + readLock.lock(); + try { + if (logger.isDebugEnabled()) { + final long byteToMbDivisor = 1024L * 1024L; + final QueueSize unacknowledged = unacknowledgedSizeRef.get(); + + logger.debug("Total Queue Size: ActiveQueue={}/{} MB, Swap Queue={}/{} MB, Unacknowledged={}/{} MB", + activeQueue.size(), activeQueueContentSize / byteToMbDivisor, + swappedRecordCount, swappedContentSize / byteToMbDivisor, + unacknowledged.getObjectCount(), unacknowledged.getByteCount() / byteToMbDivisor); + } + + return swapQueue.size(); + } finally { + readLock.unlock("getSwapQueueSize"); + } + } + + private boolean isLaterThan(final Long maxAge) { + if (maxAge == null) { + return false; + } + return maxAge < System.currentTimeMillis(); + } + + private Long getExpirationDate(final FlowFile flowFile, final long expirationMillis) { + if (flowFile == null) { + return null; + } + if (expirationMillis <= 0) { + return null; + } else { + final long entryDate = flowFile.getEntryDate(); + final long expirationDate = entryDate + expirationMillis; + return expirationDate; + } + } + + @Override + public FlowFileRecord poll(final Set expiredRecords) { + FlowFileRecord flowFile = null; + + // First check if we have any records Pre-Fetched. + final long expirationMillis = flowFileExpirationMillis.get(); + final PreFetch preFetch = preFetchRef.get(); + if (preFetch != null) { + if (preFetch.isExpired()) { + requeueExpiredPrefetch(preFetch); + } else { + while (true) { + final FlowFileRecord next = preFetch.nextRecord(); + if (next == null) { + break; + } + + if (isLaterThan(getExpirationDate(next, expirationMillis))) { + expiredRecords.add(next); + continue; + } + + updateUnacknowledgedSize(1, next.getSize()); + return next; + } + + preFetchRef.compareAndSet(preFetch, null); + } + } + + writeLock.lock(); + try { + flowFile = doPoll(expiredRecords, expirationMillis); + return flowFile; + } finally { + activeQueueSizeRef.set(activeQueue.size()); + writeLock.unlock("poll(Set)"); + + if (flowFile != null) { + updateUnacknowledgedSize(1, flowFile.getSize()); + } + } + } + + private FlowFileRecord doPoll(final Set expiredRecords, final long expirationMillis) { + FlowFileRecord flowFile; + boolean isExpired; + + migrateSwapToActive(); + boolean queueFullAtStart = queueFullRef.get(); + + do { + flowFile = this.activeQueue.poll(); + + isExpired = isLaterThan(getExpirationDate(flowFile, expirationMillis)); + if (isExpired) { + expiredRecords.add(flowFile); + if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) { + activeQueueContentSize -= flowFile.getSize(); + break; + } + } else if (flowFile != null && flowFile.isPenalized()) { + this.activeQueue.add(flowFile); + flowFile = null; + break; + } + + if (flowFile != null) { + activeQueueContentSize -= flowFile.getSize(); + } + } while (isExpired); + + // if at least 1 FlowFile was expired & the queue was full before we started, then + // we need to determine whether or not the queue is full again. If no FlowFile was expired, + // then the queue will still be full until the appropriate #acknowledge method is called. + if (queueFullAtStart && !expiredRecords.isEmpty()) { + queueFullRef.set(determineIfFull()); + } + + if (incrementPollCount()) { + prefetch(); + } + return isExpired ? null : flowFile; + } + + @Override + public List poll(int maxResults, final Set expiredRecords) { + final List records = new ArrayList<>(Math.min(1024, maxResults)); + + // First check if we have any records Pre-Fetched. + final long expirationMillis = flowFileExpirationMillis.get(); + final PreFetch preFetch = preFetchRef.get(); + if (preFetch != null) { + if (preFetch.isExpired()) { + requeueExpiredPrefetch(preFetch); + } else { + long totalSize = 0L; + for (int i = 0; i < maxResults; i++) { + final FlowFileRecord next = preFetch.nextRecord(); + if (next == null) { + break; + } + + if (isLaterThan(getExpirationDate(next, expirationMillis))) { + expiredRecords.add(next); + continue; + } + + records.add(next); + totalSize += next.getSize(); + } + + // If anything was prefetched, use what we have. + if (!records.isEmpty()) { + updateUnacknowledgedSize(records.size(), totalSize); + return records; + } + + preFetchRef.compareAndSet(preFetch, null); + } + } + + writeLock.lock(); + try { + doPoll(records, maxResults, expiredRecords); + } finally { + activeQueueSizeRef.set(activeQueue.size()); + writeLock.unlock("poll(int, Set)"); + } + return records; + } + + private void doPoll(final List records, int maxResults, final Set expiredRecords) { + migrateSwapToActive(); + + final boolean queueFullAtStart = queueFullRef.get(); + + final long bytesDrained = drainQueue(activeQueue, records, maxResults, expiredRecords); + + long expiredBytes = 0L; + for (final FlowFileRecord record : expiredRecords) { + expiredBytes += record.getSize(); + } + + activeQueueContentSize -= bytesDrained; + updateUnacknowledgedSize(records.size(), bytesDrained - expiredBytes); + + // if at least 1 FlowFile was expired & the queue was full before we started, then + // we need to determine whether or not the queue is full again. If no FlowFile was expired, + // then the queue will still be full until the appropriate #acknowledge method is called. + if (queueFullAtStart && !expiredRecords.isEmpty()) { + queueFullRef.set(determineIfFull()); + } + + if (incrementPollCount()) { + prefetch(); + } + } + + /** + * If there are FlowFiles waiting on the swap queue, move them to the active + * queue until we meet our threshold. This prevents us from having to swap + * them to disk & then back out. + * + * This method MUST be called with the writeLock held. + */ + private void migrateSwapToActive() { + // Migrate as many FlowFiles as we can from the Swap Queue to the Active Queue, so that we don't + // have to swap them out & then swap them back in. + // If we don't do this, we could get into a situation where we have potentially thousands of FlowFiles + // sitting on the Swap Queue but not getting processed because there aren't enough to be swapped out. + // In particular, this can happen if the queue is typically filled with surges. + // For example, if the queue has 25,000 FlowFiles come in, it may process 20,000 of them and leave + // 5,000 sitting on the Swap Queue. If it then takes an hour for an additional 5,000 FlowFiles to come in, + // those FlowFiles sitting on the Swap Queue will sit there for an hour, waiting to be swapped out and + // swapped back in again. + // Calling this method when records are polled prevents this condition by migrating FlowFiles from the + // Swap Queue to the Active Queue. However, we don't do this if there are FlowFiles already swapped out + // to disk, because we want them to be swapped back in in the same order that they were swapped out. + + // this is the most common condition (nothing is swapped out), so do the check first and avoid the expense + // of other checks for 99.999% of the cases. + if (swappedRecordCount == 0 && swapQueue.isEmpty()) { + return; + } + + if (swappedRecordCount > swapQueue.size()) { + // we already have FlowFiles swapped out, so we won't migrate the queue; we will wait for + // an external process to swap FlowFiles back in. + return; + } + + final Iterator swapItr = swapQueue.iterator(); + while (activeQueue.size() < swapThreshold && swapItr.hasNext()) { + final FlowFileRecord toMigrate = swapItr.next(); + activeQueue.add(toMigrate); + activeQueueContentSize += toMigrate.getSize(); + swappedContentSize -= toMigrate.getSize(); + swappedRecordCount--; + + swapItr.remove(); + } + + if (swappedRecordCount == 0) { + swapMode = false; + } + } + + @Override + public long drainQueue(final Queue sourceQueue, final List destination, int maxResults, final Set expiredRecords) { + long drainedSize = 0L; + FlowFileRecord pulled = null; + + final long expirationMillis = this.flowFileExpirationMillis.get(); + while (destination.size() < maxResults && (pulled = sourceQueue.poll()) != null) { + if (isLaterThan(getExpirationDate(pulled, expirationMillis))) { + expiredRecords.add(pulled); + if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) { + break; + } + } else { + if (pulled.isPenalized()) { + sourceQueue.add(pulled); + break; + } + destination.add(pulled); + } + drainedSize += pulled.getSize(); + } + return drainedSize; + } + + @Override + public List poll(final FlowFileFilter filter, final Set expiredRecords) { + writeLock.lock(); + try { + migrateSwapToActive(); + if (activeQueue.isEmpty()) { + return Collections.emptyList(); + } + + final long expirationMillis = this.flowFileExpirationMillis.get(); + final boolean queueFullAtStart = queueFullRef.get(); + + final List selectedFlowFiles = new ArrayList<>(); + final List unselected = new ArrayList<>(); + + while (true) { + FlowFileRecord flowFile = this.activeQueue.poll(); + if (flowFile == null) { + break; + } + + final boolean isExpired = isLaterThan(getExpirationDate(flowFile, expirationMillis)); + if (isExpired) { + expiredRecords.add(flowFile); + activeQueueContentSize -= flowFile.getSize(); + + if (expiredRecords.size() >= MAX_EXPIRED_RECORDS_PER_ITERATION) { + break; + } else { + continue; + } + } else if (flowFile.isPenalized()) { + this.activeQueue.add(flowFile); + flowFile = null; + break; // just stop searching because the rest are all penalized. + } + + final FlowFileFilterResult result = filter.filter(flowFile); + if (result.isAccept()) { + activeQueueContentSize -= flowFile.getSize(); + + updateUnacknowledgedSize(1, flowFile.getSize()); + selectedFlowFiles.add(flowFile); + } else { + unselected.add(flowFile); + } + + if (!result.isContinue()) { + break; + } + } + + this.activeQueue.addAll(unselected); + + // if at least 1 FlowFile was expired & the queue was full before we started, then + // we need to determine whether or not the queue is full again. If no FlowFile was expired, + // then the queue will still be full until the appropriate #acknowledge method is called. + if (queueFullAtStart && !expiredRecords.isEmpty()) { + queueFullRef.set(determineIfFull()); + } + + return selectedFlowFiles; + } finally { + activeQueueSizeRef.set(activeQueue.size()); + writeLock.unlock("poll(Filter, Set)"); + } + } + + private static final class Prioritizer implements Comparator, Serializable { + + private static final long serialVersionUID = 1L; + private final transient List prioritizers = new ArrayList<>(); + + private Prioritizer(final List priorities) { + if (null != priorities) { + prioritizers.addAll(priorities); + } + } + + @Override + public int compare(final FlowFileRecord f1, final FlowFileRecord f2) { + int returnVal = 0; + final boolean f1Penalized = f1.isPenalized(); + final boolean f2Penalized = f2.isPenalized(); + + if (f1Penalized && !f2Penalized) { + return 1; + } else if (!f1Penalized && f2Penalized) { + return -1; + } + + if (f1Penalized && f2Penalized) { + if (f1.getPenaltyExpirationMillis() < f2.getPenaltyExpirationMillis()) { + return -1; + } else if (f1.getPenaltyExpirationMillis() > f2.getPenaltyExpirationMillis()) { + return 1; + } + } + + if (!prioritizers.isEmpty()) { + for (final FlowFilePrioritizer prioritizer : prioritizers) { + returnVal = prioritizer.compare(f1, f2); + if (returnVal != 0) { + return returnVal; + } + } + } + + final ContentClaim claim1 = f1.getContentClaim(); + final ContentClaim claim2 = f2.getContentClaim(); + + // put the one without a claim first + if (claim1 == null && claim2 != null) { + return -1; + } else if (claim1 != null && claim2 == null) { + return 1; + } else if (claim1 != null && claim2 != null) { + final int claimComparison = claim1.compareTo(claim2); + if (claimComparison != 0) { + return claimComparison; + } + + final int claimOffsetComparison = Long.compare(f1.getContentClaimOffset(), f2.getContentClaimOffset()); + if (claimOffsetComparison != 0) { + return claimOffsetComparison; + } + } + + return Long.compare(f1.getId(), f2.getId()); + } + } + + @Override + public String getFlowFileExpiration() { + return flowFileExpirationPeriod.get(); + } + + @Override + public int getFlowFileExpiration(final TimeUnit timeUnit) { + return (int) timeUnit.convert(flowFileExpirationMillis.get(), TimeUnit.MILLISECONDS); + } + + @Override + public void setFlowFileExpiration(final String flowExpirationPeriod) { + final long millis = FormatUtils.getTimeDuration(flowExpirationPeriod, TimeUnit.MILLISECONDS); + if (millis < 0) { + throw new IllegalArgumentException("FlowFile Expiration Period must be positive"); + } + this.flowFileExpirationPeriod.set(flowExpirationPeriod); + this.flowFileExpirationMillis.set(millis); + } + + @Override + public String toString() { + return "FlowFileQueue[id=" + identifier + "]"; + } + + /** + * Lock the queue so that other threads are unable to interact with the + * queue + */ + public void lock() { + writeLock.lock(); + } + + /** + * Unlock the queue + */ + public void unlock() { + writeLock.unlock("external unlock"); + } + + private void updateUnacknowledgedSize(final int addToCount, final long addToSize) { + boolean updated = false; + + do { + QueueSize queueSize = unacknowledgedSizeRef.get(); + final QueueSize newSize = new QueueSize(queueSize.getObjectCount() + addToCount, queueSize.getByteCount() + addToSize); + updated = unacknowledgedSizeRef.compareAndSet(queueSize, newSize); + } while (!updated); + } + + private void requeueExpiredPrefetch(final PreFetch prefetch) { + if (prefetch == null) { + return; + } + + writeLock.lock(); + try { + final long contentSizeRequeued = prefetch.requeue(activeQueue); + this.activeQueueContentSize += contentSizeRequeued; + this.preFetchRef.compareAndSet(prefetch, null); + } finally { + writeLock.unlock("requeueExpiredPrefetch"); + } + } + + /** + * MUST be called with write lock held. + */ + private final AtomicReference preFetchRef = new AtomicReference<>(); + + private void prefetch() { + if (activeQueue.isEmpty()) { + return; + } + + final int numToFetch = Math.min(prefetchSize, activeQueue.size()); + + final PreFetch curPreFetch = preFetchRef.get(); + if (curPreFetch != null && curPreFetch.size().getObjectCount() > 0) { + return; + } + + final List buffer = new ArrayList<>(numToFetch); + long contentSize = 0L; + for (int i = 0; i < numToFetch; i++) { + final FlowFileRecord record = activeQueue.poll(); + if (record == null || record.isPenalized()) { + // not enough unpenalized records to pull. Put all records back and return + activeQueue.addAll(buffer); + return; + } else { + buffer.add(record); + contentSize += record.getSize(); + } + } + + activeQueueContentSize -= contentSize; + preFetchRef.set(new PreFetch(buffer)); + } + + private final TimedBuffer pollCounts = new TimedBuffer<>(TimeUnit.SECONDS, 5, new LongEntityAccess()); + + private boolean incrementPollCount() { + pollCounts.add(new TimestampedLong(1L)); + final long totalCount = pollCounts.getAggregateValue(System.currentTimeMillis() - 5000L).getValue(); + return totalCount > PREFETCH_POLL_THRESHOLD * 5; + } + + private static class PreFetch { + + private final List records; + private final AtomicInteger pointer = new AtomicInteger(0); + private final long expirationTime = System.nanoTime() + TimeUnit.SECONDS.toNanos(1L); + private final AtomicLong contentSize = new AtomicLong(0L); + + public PreFetch(final List records) { + this.records = records; + + long totalSize = 0L; + for (final FlowFileRecord record : records) { + totalSize += record.getSize(); + } + contentSize.set(totalSize); + } + + public FlowFileRecord nextRecord() { + final int nextValue = pointer.getAndIncrement(); + if (nextValue >= records.size()) { + return null; + } + + final FlowFileRecord flowFile = records.get(nextValue); + contentSize.addAndGet(-flowFile.getSize()); + return flowFile; + } + + public QueueSize size() { + final int pointerIndex = pointer.get(); + final int count = records.size() - pointerIndex; + if (count < 0) { + return new QueueSize(0, 0L); + } + + final long bytes = contentSize.get(); + return new QueueSize(count, bytes); + } + + public boolean isExpired() { + return System.nanoTime() > expirationTime; + } + + private long requeue(final Queue queue) { + // get the current pointer and prevent any other thread from accessing the rest of the elements + final int curPointer = pointer.getAndAdd(records.size()); + if (curPointer < records.size() - 1) { + final List subList = records.subList(curPointer, records.size()); + long contentSize = 0L; + for (final FlowFileRecord record : subList) { + contentSize += record.getSize(); + } + + queue.addAll(subList); + + return contentSize; + } + return 0L; + } + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java new file mode 100644 index 0000000000..52a4e40d21 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/StandardFunnel.java @@ -0,0 +1,541 @@ +/* + * 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.controller; + +import static java.util.Objects.requireNonNull; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; + +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.ConnectableType; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.Funnel; +import org.apache.nifi.connectable.Position; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.groups.ProcessGroup; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.ProcessSessionFactory; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.scheduling.SchedulingStrategy; +import org.apache.nifi.util.FormatUtils; + +import org.apache.commons.lang3.builder.ToStringBuilder; +import org.apache.commons.lang3.builder.ToStringStyle; + +public class StandardFunnel implements Funnel { + + public static final long MINIMUM_PENALIZATION_MILLIS = 0L; + public static final TimeUnit DEFAULT_TIME_UNIT = TimeUnit.MILLISECONDS; + public static final long MINIMUM_YIELD_MILLIS = 0L; + public static final long DEFAULT_YIELD_PERIOD = 1000L; + public static final TimeUnit DEFAULT_YIELD_TIME_UNIT = TimeUnit.MILLISECONDS; + + private final String identifier; + private final Set outgoingConnections; + private final List incomingConnections; + private final List relationships; + + private final AtomicReference processGroupRef; + private final AtomicReference position; + private final AtomicReference penalizationPeriod; + private final AtomicReference yieldPeriod; + private final AtomicReference schedulingPeriod; + private final AtomicReference name; + private final AtomicLong schedulingNanos; + private final AtomicBoolean lossTolerant; + private final AtomicReference scheduledState; + private final AtomicLong yieldExpiration; + + private final ReadWriteLock rwLock = new ReentrantReadWriteLock(); + private final Lock readLock = rwLock.readLock(); + private final Lock writeLock = rwLock.writeLock(); + + public StandardFunnel(final String identifier, final ProcessGroup processGroup, final ProcessScheduler scheduler) { + this.identifier = identifier; + this.processGroupRef = new AtomicReference<>(processGroup); + + outgoingConnections = new HashSet<>(); + incomingConnections = new ArrayList<>(); + + final List relationships = new ArrayList<>(); + relationships.add(Relationship.ANONYMOUS); + this.relationships = Collections.unmodifiableList(relationships); + + lossTolerant = new AtomicBoolean(false); + position = new AtomicReference<>(new Position(0D, 0D)); + scheduledState = new AtomicReference<>(ScheduledState.STOPPED); + penalizationPeriod = new AtomicReference<>("30 sec"); + yieldPeriod = new AtomicReference<>("1 sec"); + yieldExpiration = new AtomicLong(0L); + schedulingPeriod = new AtomicReference<>("0 millis"); + schedulingNanos = new AtomicLong(30000); + name = new AtomicReference<>("Funnel"); + } + + @Override + public String getIdentifier() { + return identifier; + } + + @Override + public Collection getRelationships() { + return relationships; + } + + @Override + public Relationship getRelationship(final String relationshipName) { + return (Relationship.ANONYMOUS.getName().equals(relationshipName)) ? Relationship.ANONYMOUS : null; + } + + @Override + public void addConnection(final Connection connection) throws IllegalArgumentException { + writeLock.lock(); + try { + if (!requireNonNull(connection).getSource().equals(this) && !connection.getDestination().equals(this)) { + throw new IllegalArgumentException("Cannot add a connection to a Funnel for which the Funnel is neither the Source nor the Destination"); + } + if (connection.getSource().equals(this) && connection.getDestination().equals(this)) { + throw new IllegalArgumentException("Cannot add a connection from a Funnel back to itself"); + } + + if (connection.getDestination().equals(this)) { + // don't add the connection twice. This may occur if we have a self-loop because we will be told + // to add the connection once because we are the source and again because we are the destination. + if (!incomingConnections.contains(connection)) { + incomingConnections.add(connection); + } + } + + if (connection.getSource().equals(this)) { + // don't add the connection twice. This may occur if we have a self-loop because we will be told + // to add the connection once because we are the source and again because we are the destination. + if (!outgoingConnections.contains(connection)) { + for (final Relationship relationship : connection.getRelationships()) { + if (!relationship.equals(Relationship.ANONYMOUS)) { + throw new IllegalArgumentException("No relationship with name " + relationship + " exists for Funnels"); + } + } + + outgoingConnections.add(connection); + } + } + } finally { + writeLock.unlock(); + } + } + + @Override + public boolean hasIncomingConnection() { + readLock.lock(); + try { + return !incomingConnections.isEmpty(); + } finally { + readLock.unlock(); + } + } + + @Override + public void updateConnection(final Connection connection) throws IllegalStateException { + if (requireNonNull(connection).getSource().equals(this)) { + writeLock.lock(); + try { + if (!outgoingConnections.remove(connection)) { + throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port"); + } + outgoingConnections.add(connection); + } finally { + writeLock.unlock(); + } + } + + if (connection.getDestination().equals(this)) { + writeLock.lock(); + try { + if (!incomingConnections.remove(connection)) { + throw new IllegalStateException("No Connection with ID " + connection.getIdentifier() + " is currently registered with this Port"); + } + incomingConnections.add(connection); + } finally { + writeLock.unlock(); + } + } + } + + @Override + public void removeConnection(final Connection connection) throws IllegalArgumentException, IllegalStateException { + writeLock.lock(); + try { + if (!requireNonNull(connection).getSource().equals(this)) { + final boolean existed = incomingConnections.remove(connection); + if (!existed) { + throw new IllegalStateException("The given connection is not currently registered for this ProcessorNode"); + } + return; + } + + final boolean removed = outgoingConnections.remove(connection); + if (!removed) { + throw new IllegalStateException(connection + " is not registered with " + this); + } + } finally { + writeLock.unlock(); + } + } + + @Override + public Set getConnections() { + readLock.lock(); + try { + return Collections.unmodifiableSet(outgoingConnections); + } finally { + readLock.unlock(); + } + } + + @Override + public Set getConnections(final Relationship relationship) { + readLock.lock(); + try { + if (relationship.equals(Relationship.ANONYMOUS)) { + return Collections.unmodifiableSet(outgoingConnections); + } + + throw new IllegalArgumentException("No relationship with name " + relationship.getName() + " exists for Funnels"); + } finally { + readLock.unlock(); + } + } + + @Override + public List getIncomingConnections() { + readLock.lock(); + try { + return new ArrayList<>(incomingConnections); + } finally { + readLock.unlock(); + } + } + + @Override + public Position getPosition() { + return position.get(); + } + + @Override + public void setPosition(Position position) { + this.position.set(position); + } + + @Override + public String getName() { + return name.get(); + } + + /** + * Throws {@link UnsupportedOperationException} + * + * @param name + */ + @Override + public void setName(final String name) { + throw new UnsupportedOperationException(); + } + + @Override + public String getComments() { + return ""; + } + + @Override + public void setComments(final String comments) { + throw new UnsupportedOperationException(); + } + + @Override + public ProcessGroup getProcessGroup() { + return processGroupRef.get(); + } + + @Override + public void setProcessGroup(final ProcessGroup group) { + processGroupRef.set(group); + } + + @Override + public boolean isAutoTerminated(Relationship relationship) { + return false; + } + + @Override + public boolean isRunning() { + return isRunning(this); + } + + private boolean isRunning(final Connectable source) { + return getScheduledState() == ScheduledState.RUNNING; + } + + @Override + public boolean isTriggerWhenEmpty() { + return false; + } + + @Override + public ScheduledState getScheduledState() { + return scheduledState.get(); + } + + @Override + public boolean isLossTolerant() { + return lossTolerant.get(); + } + + @Override + public void setLossTolerant(final boolean lossTolerant) { + this.lossTolerant.set(lossTolerant); + } + + @Override + public String toString() { + return new ToStringBuilder(this, ToStringStyle.SHORT_PREFIX_STYLE).append("id", getIdentifier()).toString(); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSessionFactory sessionFactory) throws ProcessException { + final ProcessSession session = sessionFactory.createSession(); + + try { + onTrigger(context, session); + session.commit(); + } catch (final ProcessException e) { + session.rollback(); + throw e; + } catch (final Throwable t) { + session.rollback(); + throw new RuntimeException(t); + } + } + + private void onTrigger(final ProcessContext context, final ProcessSession session) { + readLock.lock(); + try { + Set available = session.getAvailableRelationships(); + int transferred = 0; + while (!available.isEmpty()) { + final List flowFiles = session.get(10); + if (flowFiles.isEmpty()) { + break; + } + + transferred += flowFiles.size(); + session.transfer(flowFiles, Relationship.ANONYMOUS); + session.commit(); + available = session.getAvailableRelationships(); + } + + if (transferred == 0) { + context.yield(); + } + } finally { + readLock.unlock(); + } + } + + /** + * Has no effect + */ + @Override + public void setMaxConcurrentTasks(int taskCount) { + } + + @Override + public int getMaxConcurrentTasks() { + return 1; + } + + @Override + public void setScheduledState(final ScheduledState scheduledState) { + this.scheduledState.set(scheduledState); + } + + @Override + public ConnectableType getConnectableType() { + return ConnectableType.FUNNEL; + } + + @Override + @SuppressWarnings("unchecked") + public Collection getValidationErrors() { + return Collections.EMPTY_LIST; + } + + /** + * Updates the amount of time that this processor should avoid being + * scheduled when the processor calls + * {@link nifi.processor.ProcessContext#yield() ProcessContext.yield()} + * + * @param yieldPeriod + */ + @Override + public void setYieldPeriod(final String yieldPeriod) { + final long yieldMillis = FormatUtils.getTimeDuration(requireNonNull(yieldPeriod), TimeUnit.MILLISECONDS); + if (yieldMillis < 0) { + throw new IllegalArgumentException("Yield duration must be positive"); + } + this.yieldPeriod.set(yieldPeriod); + } + + /** + * @param schedulingPeriod + */ + @Override + public void setScheduldingPeriod(final String schedulingPeriod) { + final long schedulingNanos = FormatUtils.getTimeDuration(requireNonNull(schedulingPeriod), TimeUnit.NANOSECONDS); + if (schedulingNanos < 0) { + throw new IllegalArgumentException("Scheduling Period must be positive"); + } + + this.schedulingPeriod.set(schedulingPeriod); + this.schedulingNanos.set(Math.max(MINIMUM_SCHEDULING_NANOS, schedulingNanos)); + } + + @Override + public long getPenalizationPeriod(final TimeUnit timeUnit) { + return FormatUtils.getTimeDuration(getPenalizationPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit); + } + + @Override + public String getPenalizationPeriod() { + return penalizationPeriod.get(); + } + + /** + * Causes the processor not to be scheduled for some period of time. This + * duration can be obtained and set via the + * {@link #getYieldPeriod(TimeUnit)} and + * {@link #setYieldPeriod(long, TimeUnit)} methods. + */ + @Override + public void yield() { + final long yieldMillis = getYieldPeriod(TimeUnit.MILLISECONDS); + yieldExpiration.set(Math.max(yieldExpiration.get(), System.currentTimeMillis() + yieldMillis)); + } + + @Override + public long getYieldExpiration() { + return yieldExpiration.get(); + } + + @Override + public String getSchedulingPeriod() { + return schedulingPeriod.get(); + } + + @Override + public void setPenalizationPeriod(final String penalizationPeriod) { + this.penalizationPeriod.set(penalizationPeriod); + } + + @Override + public String getYieldPeriod() { + return yieldPeriod.get(); + } + + @Override + public long getYieldPeriod(final TimeUnit timeUnit) { + return FormatUtils.getTimeDuration(getYieldPeriod(), timeUnit == null ? DEFAULT_TIME_UNIT : timeUnit); + } + + @Override + public long getSchedulingPeriod(final TimeUnit timeUnit) { + return timeUnit.convert(schedulingNanos.get(), TimeUnit.NANOSECONDS); + } + + @Override + public boolean isSideEffectFree() { + return true; + } + + @Override + public void verifyCanDelete(boolean ignoreConnections) throws IllegalStateException { + if (ignoreConnections) { + return; + } + + readLock.lock(); + try { + for (final Connection connection : outgoingConnections) { + connection.verifyCanDelete(); + } + + for (final Connection connection : incomingConnections) { + if (connection.getSource().equals(this)) { + connection.verifyCanDelete(); + } else { + throw new IllegalStateException(this + " is the destination of another component"); + } + } + } finally { + readLock.unlock(); + } + } + + @Override + public void verifyCanDelete() { + verifyCanDelete(false); + } + + @Override + public void verifyCanStart() { + } + + @Override + public void verifyCanStop() { + } + + @Override + public void verifyCanUpdate() { + } + + @Override + public void verifyCanEnable() { + } + + @Override + public void verifyCanDisable() { + } + + @Override + public SchedulingStrategy getSchedulingStrategy() { + return SchedulingStrategy.EVENT_DRIVEN; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.java new file mode 100644 index 0000000000..df3c251c85 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/ValidationContextFactory.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.controller; + +import java.util.Map; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; + +public interface ValidationContextFactory { + + ValidationContext newValidationContext(Map properties, String annotationData); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.java new file mode 100644 index 0000000000..2f436003d3 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/WorkerQueue.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.controller; + +import java.util.concurrent.TimeUnit; + +import org.apache.nifi.connectable.Connectable; + +public interface WorkerQueue { + + EventBasedWorker poll(long timeout, TimeUnit timeUnit); + + void offer(Connectable worker); + + void setClustered(boolean clustered); + + void setPrimary(boolean primary); + + void suspendWork(Connectable worker); + + void resumeWork(Connectable worker); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java new file mode 100644 index 0000000000..368ed1b992 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/CommunicationsException.java @@ -0,0 +1,40 @@ +/* + * 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.controller.exception; + +import java.io.IOException; + +public class CommunicationsException extends IOException { + + private static final long serialVersionUID = 142343242323423L; + + public CommunicationsException() { + super(); + } + + public CommunicationsException(final Throwable cause) { + super(cause); + } + + public CommunicationsException(final String explanation) { + super(explanation); + } + + public CommunicationsException(final String explanation, final Throwable cause) { + super(explanation, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java new file mode 100644 index 0000000000..0ff68b035b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceAlreadyExistsException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.exception; + +public class ControllerServiceAlreadyExistsException extends RuntimeException { + + private static final long serialVersionUID = -544424320587059277L; + + /** + * Constructs a default exception + * @param id + */ + public ControllerServiceAlreadyExistsException(final String id) { + super("A Controller Service already exists with ID " + id); + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java new file mode 100644 index 0000000000..4cdbe5406f --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ControllerServiceNotFoundException.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.exception; + +public class ControllerServiceNotFoundException extends RuntimeException { + + private static final long serialVersionUID = -544424320587059277L; + + /** + * Constructs a default exception + */ + public ControllerServiceNotFoundException() { + super(); + } + + /** + * @param message + */ + public ControllerServiceNotFoundException(String message) { + super(message); + } + + /** + * @param cause + */ + public ControllerServiceNotFoundException(Throwable cause) { + super(cause); + } + + /** + * @param message + * @param cause + */ + public ControllerServiceNotFoundException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.java new file mode 100644 index 0000000000..c4aba442d6 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorInstantiationException.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.controller.exception; + +public class ProcessorInstantiationException extends Exception { + + private static final long serialVersionUID = 189273489L; + + public ProcessorInstantiationException(final String className, final Throwable t) { + super(className, t); + } + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java new file mode 100644 index 0000000000..5acca16552 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/exception/ProcessorLifeCycleException.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.exception; + +public class ProcessorLifeCycleException extends RuntimeException { + + private static final long serialVersionUID = 8392341500511490941L; + + public ProcessorLifeCycleException(final String message, final Throwable t) { + super(message, t); + } + + public ProcessorLifeCycleException(final Throwable t) { + super(t); + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java new file mode 100644 index 0000000000..97c44b5505 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/label/Label.java @@ -0,0 +1,48 @@ +/* + * 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.controller.label; + +import java.util.Map; + +import org.apache.nifi.connectable.Position; +import org.apache.nifi.connectable.Size; +import org.apache.nifi.groups.ProcessGroup; + +public interface Label { + + String getIdentifier(); + + Position getPosition(); + + void setPosition(Position position); + + Map getStyle(); + + void setStyle(Map style); + + Size getSize(); + + void setSize(Size size); + + ProcessGroup getProcessGroup(); + + void setProcessGroup(ProcessGroup group); + + String getValue(); + + void setValue(String value); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java new file mode 100644 index 0000000000..ced6ff975b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/reporting/ReportingTaskInstantiationException.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.reporting; + +public class ReportingTaskInstantiationException extends Exception { + + private static final long serialVersionUID = 189234789237L; + + public ReportingTaskInstantiationException(final String className, final Throwable t) { + super(className, t); + } + + public ReportingTaskInstantiationException(final String message) { + super(message); + } + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java new file mode 100644 index 0000000000..6ce7ba6923 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/ContentNotFoundException.java @@ -0,0 +1,48 @@ +/* + * 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.controller.repository; + +import org.apache.nifi.controller.repository.claim.ContentClaim; + +/** + * + * @author none + */ +public class ContentNotFoundException extends RuntimeException { + + private static final long serialVersionUID = 19048239082L; + private final transient ContentClaim claim; + + public ContentNotFoundException(final ContentClaim claim) { + super("Could not find content for " + claim); + this.claim = claim; + } + + public ContentNotFoundException(final ContentClaim claim, final Throwable t) { + super("Could not find content for " + claim, t); + this.claim = claim; + } + + public ContentNotFoundException(final ContentClaim claim, final String message) { + super("Could not find content for " + claim + ": " + message); + this.claim = claim; + } + + public ContentClaim getMissingClaim() { + return claim; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java new file mode 100644 index 0000000000..de231ed1b5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/CounterRepository.java @@ -0,0 +1,34 @@ +/* + * 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.controller.repository; + +import java.util.List; + +import org.apache.nifi.controller.Counter; + +public interface CounterRepository { + + void adjustCounter(String counterContext, String name, long delta); + + Counter getCounter(String counterContext, String name); + + List getCounters(); + + List getCounters(String counterContext); + + Counter resetCounter(String identifier); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java new file mode 100644 index 0000000000..f07a5308dd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEvent.java @@ -0,0 +1,54 @@ +/* + * 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.controller.repository; + +public interface FlowFileEvent { + + String getComponentIdentifier(); + + int getFlowFilesIn(); + + int getFlowFilesOut(); + + int getFlowFilesRemoved(); + + long getContentSizeIn(); + + long getContentSizeOut(); + + long getContentSizeRemoved(); + + long getBytesRead(); + + long getBytesWritten(); + + long getProcessingNanoseconds(); + + long getAverageLineageMillis(); + + long getAggregateLineageMillis(); + + int getFlowFilesReceived(); + + long getBytesReceived(); + + int getFlowFilesSent(); + + long getBytesSent(); + + int getInvocations(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java new file mode 100644 index 0000000000..2eb3caf760 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/FlowFileEventRepository.java @@ -0,0 +1,50 @@ +/* + * 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.controller.repository; + +import java.io.Closeable; +import java.io.IOException; + +/** + * + * @author none + */ +public interface FlowFileEventRepository extends Closeable { + + /** + * Updates the repository to include a new FlowFile processing event + * + * @param event + * @throws java.io.IOException + */ + void updateRepository(FlowFileEvent event) throws IOException; + + /** + * Returns a report of processing activity since the given time + * @param sinceEpochMillis + * @return + */ + RepositoryStatusReport reportTransferEvents(long sinceEpochMillis); + + /** + * Causes any flow file events of the given entry age in epoch milliseconds + * or older to be purged from the repository + * + * @param cutoffEpochMilliseconds + */ + void purgeTransferEvents(long cutoffEpochMilliseconds); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java new file mode 100644 index 0000000000..e434905d46 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/repository/RepositoryStatusReport.java @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.controller.repository; + +import java.util.Map; + +public interface RepositoryStatusReport { + + void addReportEntry(FlowFileEvent entry); + + Map getReportEntries(); + + FlowFileEvent getReportEntry(String componentId); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java new file mode 100644 index 0000000000..6f9c237222 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceNode.java @@ -0,0 +1,40 @@ +/* + * 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.controller.service; + +import org.apache.nifi.controller.Availability; +import org.apache.nifi.controller.ConfiguredComponent; +import org.apache.nifi.controller.ControllerService; + +public interface ControllerServiceNode extends ConfiguredComponent { + + ControllerService getControllerService(); + + Availability getAvailability(); + + void setAvailability(Availability availability); + + boolean isDisabled(); + + void setDisabled(boolean disabled); + + ControllerServiceReference getReferences(); + + void addReference(ConfiguredComponent referringComponent); + + void removeReference(ConfiguredComponent referringComponent); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java new file mode 100644 index 0000000000..35a255d8dd --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceProvider.java @@ -0,0 +1,47 @@ +/* + * 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.controller.service; + +import java.util.Map; + +import org.apache.nifi.controller.ControllerServiceLookup; + +/** + * + */ +public interface ControllerServiceProvider extends ControllerServiceLookup { + + /** + * Gets the controller service for the specified identifier. Returns null if + * the identifier does not match a known service. + * + * @param type + * @param id + * @param properties + * @return + */ + ControllerServiceNode createControllerService(String type, String id, Map properties); + + /** + * Gets the controller service node for the specified identifier. Returns + * null if the identifier does not match a known service + * + * @param id + * @return + */ + ControllerServiceNode getControllerServiceNode(String id); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java new file mode 100644 index 0000000000..5cb676f494 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/controller/service/ControllerServiceReference.java @@ -0,0 +1,50 @@ +/* + * 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.controller.service; + +import java.util.Set; + +import org.apache.nifi.controller.ConfiguredComponent; + +/** + * Provides a collection of components that are referencing a Controller Service + */ +public interface ControllerServiceReference { + + /** + * Returns the component that is being referenced + * + * @return + */ + ControllerServiceNode getReferencedComponent(); + + /** + * Returns a {@link Set} of all components that are referencing this + * Controller Service + * + * @return + */ + Set getReferencingComponents(); + + /** + * Returns a {@link Set} of all Processors and Reporting Tasks that are + * referencing the Controller Service and are running, in addition to all + * + * @return + */ + Set getRunningReferences(); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java new file mode 100644 index 0000000000..d1d5e5b76b --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java @@ -0,0 +1,52 @@ +/* + * 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.events; + +import java.util.concurrent.atomic.AtomicLong; +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.reporting.Bulletin; + +/** + * + */ +public final class BulletinFactory { + + private static final AtomicLong currentId = new AtomicLong(0); + + public static Bulletin createBulletin(final Connectable connectable, final String category, final String severity, final String message) { + return BulletinFactory.createBulletin(connectable.getProcessGroup().getIdentifier(), connectable.getIdentifier(), connectable.getName(), category, severity, message); + } + + public static Bulletin createBulletin(final String groupId, final String sourceId, final String sourceName, final String category, final String severity, final String message) { + final Bulletin bulletin = new ComponentBulletin(currentId.getAndIncrement()); + bulletin.setGroupId(groupId); + bulletin.setSourceId(sourceId); + bulletin.setSourceName(sourceName); + bulletin.setCategory(category); + bulletin.setLevel(severity); + bulletin.setMessage(message); + return bulletin; + } + + public static Bulletin createBulletin(final String category, final String severity, final String message) { + final Bulletin bulletin = new SystemBulletin(currentId.getAndIncrement()); + bulletin.setCategory(category); + bulletin.setLevel(severity); + bulletin.setMessage(message); + return bulletin; + } +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.java new file mode 100644 index 0000000000..9846cf2bc5 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/BulletinProcessingStrategy.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.events; + +import org.apache.nifi.reporting.Bulletin; + +/** + * + */ +public interface BulletinProcessingStrategy { + + void update(Bulletin bulletin); +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java new file mode 100644 index 0000000000..23c4cdb0d9 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/ComponentBulletin.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.events; + +import org.apache.nifi.reporting.Bulletin; + +/** + * + */ +public class ComponentBulletin extends Bulletin { + + ComponentBulletin(final long id) { + super(id); + } + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java new file mode 100644 index 0000000000..f97dc4663e --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/events/SystemBulletin.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.events; + +import org.apache.nifi.reporting.Bulletin; + +/** + * + */ +public class SystemBulletin extends Bulletin { + + SystemBulletin(final long id) { + super(id); + } + +} diff --git a/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java new file mode 100644 index 0000000000..61be59c349 --- /dev/null +++ b/nar-bundles/framework-bundle/framework/core-api/src/main/java/org/apache/nifi/groups/ProcessGroup.java @@ -0,0 +1,723 @@ +/* + * 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.groups; + +import java.util.Collection; +import java.util.List; +import java.util.Set; + +import org.apache.nifi.connectable.Connectable; +import org.apache.nifi.connectable.Connection; +import org.apache.nifi.connectable.Funnel; +import org.apache.nifi.connectable.Port; +import org.apache.nifi.connectable.Position; +import org.apache.nifi.controller.ProcessorNode; +import org.apache.nifi.controller.Snippet; +import org.apache.nifi.controller.label.Label; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.Processor; + +/** + *

+ * ProcessGroup objects are containers for processing entities, such as + * {@link Processor}s, {@link Port}s, and other {@link ProcessGroup}s. + *

+ * + *

+ * MUST BE THREAD-SAFE

+ */ +public interface ProcessGroup { + + /** + * @return a reference to this ProcessGroup's parent. This will be + * null if and only if this is the root group. + */ + ProcessGroup getParent(); + + /** + * Updates the ProcessGroup to point to a new parent + * + * @param group + */ + void setParent(ProcessGroup group); + + /** + * @return the ID of the ProcessGroup + */ + String getIdentifier(); + + /** + * @return the name of the ProcessGroup + */ + String getName(); + + /** + * Updates the name of this ProcessGroup. + * + * @param name + */ + void setName(String name); + + /** + * Updates the position of where this ProcessGroup is located in the graph + */ + void setPosition(Position position); + + /** + * Returns the position of where this ProcessGroup is located in the graph + * + * @return + */ + Position getPosition(); + + /** + * @return the user-set comments about this ProcessGroup, or + * null if no comments have been set + */ + String getComments(); + + /** + * Updates the comments for this ProcessGroup + * + * @param comments + */ + void setComments(String comments); + + /** + * Returns the counts for this ProcessGroup + * + * @return + */ + ProcessGroupCounts getCounts(); + + /** + * Starts all Processors, Local Ports, and Funnels that are directly within + * this group and any child ProcessGroups, except for those that are + * disabled. + */ + void startProcessing(); + + /** + * Stops all Processors, Local Ports, and Funnels that are directly within + * this group and child ProcessGroups, except for those that are disabled. + */ + void stopProcessing(); + + /** + * Starts the given Processor + * + * @param processor the processor to start + * @throws IllegalStateException if the processor is not valid, or is + * already running + */ + void enableProcessor(ProcessorNode processor); + + /** + * Starts the given Input Port + * + * @param port + */ + void enableInputPort(Port port); + + /** + * Starts the given Output Port + * + * @param port + */ + void enableOutputPort(Port port); + + /** + * Starts the given Funnel + * + * @param funnel + */ + void enableFunnel(Funnel funnel); + + /** + * Starts the given Processor + * + * @param processor the processor to start + * @throws IllegalStateException if the processor is not valid, or is + * already running + */ + void startProcessor(ProcessorNode processor); + + /** + * Starts the given Input Port + * + * @param port + */ + void startInputPort(Port port); + + /** + * Starts the given Output Port + * + * @param port + */ + void startOutputPort(Port port); + + /** + * Starts the given Funnel + * + * @param funnel + */ + void startFunnel(Funnel funnel); + + /** + * Stops the given Processor + * + * @param processor + */ + void stopProcessor(ProcessorNode processor); + + /** + * Stops the given Port + * + * @param processor + */ + void stopInputPort(Port port); + + /** + * Stops the given Port + * + * @param processor + */ + void stopOutputPort(Port port); + + /** + * Stops the given Funnel + * + * @param processor + */ + void stopFunnel(Funnel funnel); + + /** + * Starts the given Processor + * + * @param processor the processor to start + * @throws IllegalStateException if the processor is not valid, or is + * already running + */ + void disableProcessor(ProcessorNode processor); + + /** + * Starts the given Input Port + * + * @param port + */ + void disableInputPort(Port port); + + /** + * Starts the given Output Port + * + * @param port + */ + void disableOutputPort(Port port); + + /** + * Starts the given Funnel + * + * @param funnel + */ + void disableFunnel(Funnel funnel); + + /** + * Indicates that the Flow is being shutdown; allows cleanup of resources + * associated with processors, etc. + */ + void shutdown(); + + /** + * Returns a boolean indicating whether or not this ProcessGroup is the root + * group + * + * @return + */ + boolean isRootGroup(); + + /** + * Adds a {@link Port} to be used for transferring {@link FlowFile}s from + * external sources to {@link Processor}s and other {@link Port}s within + * this ProcessGroup. + * + * @param port + */ + void addInputPort(Port port); + + /** + * Removes a {@link Port} from this ProcessGroup's list of Input Ports. + * + * @param port the Port to remove + * @throws NullPointerException if port is null + * @throws IllegalStateException if port is not an Input Port for this + * ProcessGroup + */ + void removeInputPort(Port port); + + /** + * @return the {@link Set} of all {@link Port}s that are used by this + * ProcessGroup as Input Ports. + */ + Set getInputPorts(); + + /** + * @param id the ID of the input port + * @return the input port with the given ID, or null if it does + * not exist. + */ + Port getInputPort(String id); + + /** + * Adds a {@link Port} to be used for transferring {@link FlowFile}s to + * external sources. + * + * @param port the Port to add + */ + void addOutputPort(Port port); + + /** + * Removes a {@link Port} from this ProcessGroup's list of Output Ports. + * + * @param port the Port to remove + * @throws NullPointerException if port is null + * @throws IllegalStateException if port is not an Input Port for this + * ProcessGroup + */ + void removeOutputPort(Port port); + + /** + * @param id the ID of the output port + * @return the output port with the given ID, or null if it + * does not exist. + */ + Port getOutputPort(String id); + + /** + * @return the {@link Set} of all {@link Port}s that are used by this + * ProcessGroup as Output Ports. + */ + Set getOutputPorts(); + + /** + * Adds a reference to a ProgressGroup as a child of this. + * + * @return the newly created reference + */ + void addProcessGroup(ProcessGroup group); + + /** + * Returns the ProcessGroup whose parent is this and whose id + * is given + * + * @param id + * @return + */ + ProcessGroup getProcessGroup(String id); + + /** + * @return a {@link Set} of all Process Group References that are contained + * within this. + */ + Set getProcessGroups(); + + /** + * @param group the group to remove + * @throws NullPointerException if group is null + * @throws IllegalStateException if group is not member of this + * ProcessGroup, or the given ProcessGroup is not empty (i.e., it contains + * at least one Processor, ProcessGroup, Input Port, Output Port, or Label). + */ + void removeProcessGroup(ProcessGroup group); + + /** + * Adds the already constructed processor instance to this group + * + * @param processor the processor to add + */ + void addProcessor(ProcessorNode processor); + + /** + * Removes the given processor from this group, destroying the Processor. + * The Processor is removed from the ProcessorRegistry, and any method in + * the Processor that is annotated with the + * {@link nifi.processor.annotation.OnRemoved OnRemoved} annotation will be + * invoked. All outgoing connections will also be destroyed + * + * @param processor the Processor to remove + * @throws NullPointerException if processor is null + * @throws IllegalStateException if processor is not a member + * of this ProcessGroup, is currently running, or has any incoming + * connections. + */ + void removeProcessor(ProcessorNode processor); + + /** + * @return a {@link Collection} of all FlowFileProcessors that are contained + * within this. + */ + Set getProcessors(); + + /** + * Returns the FlowFileProcessor with the given ID. + * + * @param id the ID of the processor to retrieve + * @return the processor with the given ID + * @throws NullPointerException if id is null. + */ + ProcessorNode getProcessor(String id); + + /** + * Returns the Connectable with the given ID, or + * null if the Connectable is not a member of the + * group + * + * @param id the ID of the Connectable + * @return + */ + Connectable getConnectable(String id); + + /** + * Adds the given connection to this ProcessGroup. This method also notifies + * the Source and Destination of the Connection that the Connection has been + * established. + * + * @param connection + * @throws NullPointerException if the connection is null + * @throws IllegalStateException if the source or destination of the + * connection is not a member of this ProcessGroup or if a connection + * already exists in this ProcessGroup with the same ID + */ + void addConnection(Connection connection); + + /** + * Removes the connection from this ProcessGroup. + * + * @param connection + * @throws IllegalStateException if connection is not contained + * within this. + */ + void removeConnection(Connection connection); + + /** + * Inherits a Connection from another ProcessGroup; this does not perform + * any validation but simply notifies the ProcessGroup that it is now the + * owner of the given Connection. This is used in place of the + * {@link #addConnection(Connection)} method when moving Connections from + * one group to another because addConnection notifies both the Source and + * Destination of the Connection that the Connection has been established; + * this method does not notify either, as both the Source and Destination + * should already be aware of the Connection. + * + * @param connection + */ + void inheritConnection(Connection connection); + + /** + * @return the Connection with the given ID, or null if the + * connection does not exist. + */ + Connection getConnection(String id); + + /** + * Returns the {@link Set} of all {@link Connection}s contained within this. + * + * @return + */ + Set getConnections(); + + /** + * Returns a List of all Connections contains within this ProcessGroup and + * any child ProcessGroups. + * + * @return + */ + List findAllConnections(); + + /** + * Adds the given RemoteProcessGroup to this ProcessGroup + * + * @param remoteGroup + * + * @throws NullPointerException if the given argument is null + */ + void addRemoteProcessGroup(RemoteProcessGroup remoteGroup); + + /** + * Removes the given RemoteProcessGroup from this ProcessGroup + * + * @param remoteGroup + * @throws NullPointerException if the argument is null + * @throws IllegalStateException if the given argument does not belong to + * this ProcessGroup + */ + void removeRemoteProcessGroup(RemoteProcessGroup remoteGroup); + + /** + * Returns the RemoteProcessGroup that is the child of this ProcessGroup and + * has the given ID. If no RemoteProcessGroup can be found with the given + * ID, returns null. + * + * @param id + * @return + */ + RemoteProcessGroup getRemoteProcessGroup(String id); + + /** + * Returns a set of all RemoteProcessGroups that belong to this + * ProcessGroup. If no RemoteProcessGroup's have been added to this + * ProcessGroup, will return an empty Set. + * + * @return + */ + Set getRemoteProcessGroups(); + + /** + * Adds the given Label to this ProcessGroup + * + * @param label the label to add + * @return + * + * @throws NullPointerException if the argument is null + */ + void addLabel(Label label); + + /** + * Removes the given Label from this ProcessGroup + * + * @param label the label to remove + * @throws NullPointerException if the argument is null + * @throws IllegalStateException if the given argument does not belong to + * this ProcessGroup + */ + void removeLabel(Label label); + + /** + * Returns a set of all Labels that belong to this ProcessGroup. If no + * Labels belong to this ProcessGroup, returns an empty Set. + * + * @return + */ + Set