diff --git a/nifi-assembly/LICENSE b/nifi-assembly/LICENSE index 2191651decc0..7669f9c4b6b3 100644 --- a/nifi-assembly/LICENSE +++ b/nifi-assembly/LICENSE @@ -1760,3 +1760,36 @@ The binary distribution of this product bundles 'HdrHistogram' which is availabl CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +The binary distribution of this product bundles source from 'Google Auth Library' which is available +under a 3-Clause BSD style license: + + Copyright 2014, Google Inc. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE index 5711c657322e..61e0684039a4 100644 --- a/nifi-assembly/NOTICE +++ b/nifi-assembly/NOTICE @@ -1196,6 +1196,11 @@ The derived work is adapted from release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/io/orc/WriterImpl.java and can be found in the org.apache.hadoop.hive.ql.io.orc package + (ASLv2) Google Cloud Java Client + The following NOTICE information applies: + Google Cloud Java Client + Copyright Google Inc. All Rights Reserved. + ************************ Common Development and Distribution License 1.1 ************************ diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 177d77ca6d0a..2f798fe055c4 100755 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -423,6 +423,11 @@ language governing permissions and limitations under the License. --> nifi-tcp-nar nar + + org.apache.nifi + nifi-gcp-nar + nar + diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml new file mode 100644 index 000000000000..af67c36b4098 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/pom.xml @@ -0,0 +1,40 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-gcp-bundle + 1.2.0-SNAPSHOT + + + nifi-gcp-nar + nar + + true + true + + + + + org.apache.nifi + nifi-gcp-processors + 1.2.0-SNAPSHOT + + + + diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 000000000000..204d556de631 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,247 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. + + +APACHE NIFI SUBCOMPONENTS: + +The Apache NiFi project contains subcomponents with separate copyright +notices and license terms. Your use of the source code for the these +subcomponents is subject to the terms and conditions of the following +licenses. + + + The binary distribution of this product bundles source from 'Google Auth Library'. + The source is available under a BSD 3-Clause: + + Copyright 2014, Google Inc. All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provided that the following conditions are + met: + + * Redistributions of source code must retain the above copyright + notice, this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above + copyright notice, this list of conditions and the following disclaimer + in the documentation and/or other materials provided with the + distribution. + + * Neither the name of Google Inc. nor the names of its + contributors may be used to endorse or promote products derived from + this software without specific prior written permission. + + THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS + "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT + LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR + A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT + OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, + SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT + LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, + DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY + THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT + (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE + OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + + The binary distribution of this product bundles source from 'Google Cloud Library'. + The source is available under an Apache License, Version 2.0 \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/src/main/resources/META-INF/NOTICE new file mode 100644 index 000000000000..74ae8eebac25 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-nar/src/main/resources/META-INF/NOTICE @@ -0,0 +1,16 @@ +nifi-gcp-nar +Copyright 2015-2016 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +****************** +Apache Software License v2 +****************** + +The following binary components are provided under the Apache Software License v2 + + (ASLv2) Google Cloud Java Client + The following NOTICE information applies: + Google Cloud Java Client + Copyright Google Inc. All Rights Reserved. \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml new file mode 100644 index 000000000000..f27b91e55721 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml @@ -0,0 +1,111 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-gcp-bundle + 1.2.0-SNAPSHOT + + + nifi-gcp-processors + jar + + + true + + + + + org.apache.nifi + nifi-api + + + org.apache.nifi + nifi-processor-utils + + + org.apache.nifi + nifi-mock + test + + + org.mockito + mockito-core + 2.3.7 + + + org.slf4j + slf4j-simple + test + + + junit + junit + 4.11 + test + + + + com.google.cloud + google-cloud + 0.8.0 + + + com.google.auth + google-auth-library-oauth2-http + 0.6.0 + + + + + + + org.apache.rat + apache-rat-plugin + + + src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker + src/test/resources/mock-gcp-service-account.json + src/test/resources/mock-gcp-application-default-credentials.json + + + + + org.apache.maven.plugins + maven-failsafe-plugin + 2.19.1 + + org.apache.nifi.processors.gcp.GCPIntegrationTests + ${skipGCPIntegrationTests} + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.19.1 + + org.apache.nifi.processors.gcp.GCPIntegrationTests + + ${project.basedir}/src/test/resources/mock-gcp-application-default-credentials.json + + + + + + diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java new file mode 100644 index 000000000000..08d8b6126d6f --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.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.processors.gcp; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.HttpServiceOptions; +import com.google.cloud.Service; +import com.google.common.collect.ImmutableList; +import org.apache.nifi.annotation.lifecycle.OnScheduled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.AbstractProcessor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsService; + +import java.util.List; + +/** + * Abstract base class for gcp processors. + * + */ +public abstract class AbstractGCPProcessor< + CloudService extends Service, + CloudServiceRpc, + CloudServiceOptions extends HttpServiceOptions> extends AbstractProcessor { + + public static final PropertyDescriptor PROJECT_ID = new PropertyDescriptor + .Builder().name("gcp-project-id") + .displayName("Project ID") + .description("Google Cloud Project ID") + .required(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor RETRY_COUNT = new PropertyDescriptor + .Builder().name("gcp-retry-count") + .displayName("Number of retries") + .description("How many retry attempts should be made before routing to the failure relationship.") + .defaultValue("6") + .required(true) + .addValidator(StandardValidators.INTEGER_VALIDATOR) + .build(); + + /** + * Links to the {@link GCPCredentialsService} which provides credentials for this particular processor. + */ + public static final PropertyDescriptor GCP_CREDENTIALS_PROVIDER_SERVICE = new PropertyDescriptor.Builder() + .name("gcp-credentials-provider-service") + .name("GCP Credentials Provider Service") + .description("The Controller Service used to obtain Google Cloud Platform credentials.") + .required(true) + .identifiesControllerService(GCPCredentialsService.class) + .build(); + + + protected volatile CloudService cloudService; + + protected CloudService getCloudService() { + return cloudService; + } + + @Override + public List getSupportedPropertyDescriptors() { + return ImmutableList.of( + GCP_CREDENTIALS_PROVIDER_SERVICE, + PROJECT_ID, + RETRY_COUNT + ); + } + + + /** + * Retrieve credentials from the {@link GCPCredentialsService} attached to this processor. + * @param context the process context provided on scheduling the processor. + * @return GoogleCredentials for the processor to access. + * @see AuthCredentials + */ + private GoogleCredentials getGoogleCredentials(final ProcessContext context) { + final GCPCredentialsService gcpCredentialsService = + context.getProperty(GCP_CREDENTIALS_PROVIDER_SERVICE).asControllerService(GCPCredentialsService.class); + return gcpCredentialsService.getGoogleCredentials(); + } + + /** + * Assigns the cloud service client on scheduling. + * @param context the process context provided on scheduling the processor. + */ + @OnScheduled + public void onScheduled(ProcessContext context) { + final CloudServiceOptions options = getServiceOptions(context, getGoogleCredentials(context)); + this.cloudService = options.getService(); + } + + /** + * Builds the service-specific options as a necessary step in creating a cloud service. + * @param context the process context provided on scheduling the processor. + * @param credentials valid GoogleCredentials retrieved by the controller service. + * @return CloudServiceOptions which can be initialized into a cloud service. + * @see ServiceOptions + */ + protected abstract CloudServiceOptions getServiceOptions(ProcessContext context, GoogleCredentials credentials); +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.java new file mode 100644 index 000000000000..bd99dd83ee54 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialPropertyDescriptors.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.processors.gcp.credentials.factory; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.util.StandardValidators; + +/** + * Shared definitions of properties that specify various GCP credentials. + */ +public final class CredentialPropertyDescriptors { + + private CredentialPropertyDescriptors() {} + + /** + * Specifies use of Application Default Credentials + * + * @see + * Google Application Default Credentials + * + */ + public static final PropertyDescriptor USE_APPLICATION_DEFAULT_CREDENTIALS = new PropertyDescriptor.Builder() + .name("application-default-credentials") + .displayName("Use Application Default Credentials") + .expressionLanguageSupported(false) + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .sensitive(false) + .allowableValues("true", "false") + .defaultValue("false") + .description("If true, uses Google Application Default Credentials, which checks the " + + "GOOGLE_APPLICATION_CREDENTIALS environment variable for a filepath to a service account JSON " + + "key, the config generated by the gcloud sdk, the App Engine service account, and the Compute" + + " Engine service account.") + .build(); + + public static final PropertyDescriptor USE_COMPUTE_ENGINE_CREDENTIALS = new PropertyDescriptor.Builder() + .name("compute-engine-credentials") + .displayName("Use Compute Engine Credentials") + .expressionLanguageSupported(false) + .required(false) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .sensitive(false) + .allowableValues("true", "false") + .defaultValue("false") + .description("If true, uses Google Compute Engine Credentials of the Compute Engine VM Instance " + + "which NiFi is running on.") + .build(); + + /** + * Specifies use of Service Account Credentials + * + * @see + * Google Service Accounts + * + */ + public static final PropertyDescriptor SERVICE_ACCOUNT_JSON_FILE = new PropertyDescriptor.Builder() + .name("service-account-json-file") + .displayName("Service Account JSON File") + .expressionLanguageSupported(false) + .required(false) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .description("Path to a file containing a Service Account key file in JSON format.") + .build(); + + public static final PropertyDescriptor SERVICE_ACCOUNT_JSON = new PropertyDescriptor.Builder() + .name("service-account-json") + .displayName("Service Account JSON") + .expressionLanguageSupported(true) + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .description("The raw JSON containing a Service Account keyfile.") + .sensitive(true) + .build(); +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.java new file mode 100644 index 000000000000..8b9c4f4d100f --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactory.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.processors.gcp.credentials.factory; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.processors.gcp.credentials.factory.strategies.ComputeEngineCredentialsStrategy; +import org.apache.nifi.processors.gcp.credentials.factory.strategies.ExplicitApplicationDefaultCredentialsStrategy; +import org.apache.nifi.processors.gcp.credentials.factory.strategies.ImplicitApplicationDefaultCredentialsStrategy; +import org.apache.nifi.processors.gcp.credentials.factory.strategies.JsonFileServiceAccountCredentialsStrategy; +import org.apache.nifi.processors.gcp.credentials.factory.strategies.JsonStringServiceAccountCredentialsStrategy; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; + +/** + * Generates GCP credentials in the form of GoogleCredential implementations for processors + * and controller services. The factory supports a number of strategies for specifying and validating + * GCP credentials, interpreted as an ordered list of most-preferred to least-preferred. + * + * Additional strategies should implement CredentialsStrategy, then be added to the strategies list in the + * constructor. + * + * @see org.apache.nifi.processors.gcp.credentials.factory.strategies + */ +public class CredentialsFactory { + + private final List strategies = new ArrayList(); + + public CredentialsFactory() { + // Primary Credential Strategies + strategies.add(new ExplicitApplicationDefaultCredentialsStrategy()); + strategies.add(new JsonFileServiceAccountCredentialsStrategy()); + strategies.add(new JsonStringServiceAccountCredentialsStrategy()); + strategies.add(new ComputeEngineCredentialsStrategy()); + + // Implicit Default is the catch-all primary strategy + strategies.add(new ImplicitApplicationDefaultCredentialsStrategy()); + } + + public CredentialsStrategy selectPrimaryStrategy(final Map properties) { + for (CredentialsStrategy strategy : strategies) { + if (strategy.canCreatePrimaryCredential(properties)) { + return strategy; + } + } + return null; + } + + public CredentialsStrategy selectPrimaryStrategy(final ValidationContext validationContext) { + final Map properties = validationContext.getProperties(); + return selectPrimaryStrategy(properties); + } + + /** + * Validates GCP credential properties against the configured strategies to report any validation errors. + * @return Validation errors + */ + public Collection validate(final ValidationContext validationContext) { + final CredentialsStrategy selectedStrategy = selectPrimaryStrategy(validationContext); + final ArrayList validationFailureResults = new ArrayList(); + + for (CredentialsStrategy strategy : strategies) { + final Collection strategyValidationFailures = strategy.validate(validationContext, + selectedStrategy); + if (strategyValidationFailures != null) { + validationFailureResults.addAll(strategyValidationFailures); + } + } + + return validationFailureResults; + } + + /** + * Produces the AuthCredentials according to the given property set and the strategies configured in + * the factory. + * @return AuthCredentials + * + * @throws IOException if there is an issue accessing the credential files + */ + public GoogleCredentials getGoogleCredentials(final Map properties) throws IOException { + final CredentialsStrategy primaryStrategy = selectPrimaryStrategy(properties); + return primaryStrategy.getGoogleCredentials(properties); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.java new file mode 100644 index 000000000000..b2a9168c019e --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsStrategy.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.processors.gcp.credentials.factory; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; + +import java.io.IOException; +import java.util.Collection; +import java.util.Map; + +/** + * Specifies a strategy for validating and creating GCP credentials from a list of properties configured on a + * Processor, Controller Service, Reporting Service, or other component. Supports only primary credentials like + * default credentials or API keys. + */ +public interface CredentialsStrategy { + + /** + * Name of the strategy, suitable for displaying to a user in validation messages. + * @return strategy name + */ + String getName(); + + /** + * Determines if this strategy can create primary credentials using the given properties. + * @return true if primary credentials can be created + */ + boolean canCreatePrimaryCredential(Map properties); + + + /** + * Validates the properties belonging to this strategy, given the selected primary strategy. Errors may result + * from individually malformed properties, invalid combinations of properties, or inappropriate use of properties + * not consistent with the primary strategy. + * @param primaryStrategy the prevailing primary strategy + * @return validation errors + */ + Collection validate(ValidationContext validationContext, CredentialsStrategy primaryStrategy); + + /** + * Creates an AuthCredentials instance for this strategy, given the properties defined by the user. + * + * @throws IOException if the provided credentials cannot be accessed or are invalid + */ + GoogleCredentials getGoogleCredentials(Map properties) throws IOException; +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractBooleanCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractBooleanCredentialsStrategy.java new file mode 100644 index 000000000000..0f94a2affb22 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractBooleanCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialsStrategy; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; + + +/** + * Partial implementation of CredentialsStrategy to provide support for credential strategies specified by + * a single boolean property. + */ +public abstract class AbstractBooleanCredentialsStrategy extends AbstractCredentialsStrategy { + + private PropertyDescriptor strategyProperty; + + public AbstractBooleanCredentialsStrategy(String name, PropertyDescriptor strategyProperty) { + super(name, new PropertyDescriptor[]{ + strategyProperty + }); + this.strategyProperty = strategyProperty; + } + + @Override + public boolean canCreatePrimaryCredential(Map properties) { + return (properties.get(this.strategyProperty) != null + && properties.get(this.strategyProperty).equalsIgnoreCase("true")); + } + + @Override + public Collection validate(final ValidationContext validationContext, + final CredentialsStrategy primaryStrategy) { + boolean thisIsSelectedStrategy = this == primaryStrategy; + Boolean useStrategy = validationContext.getProperty(strategyProperty).asBoolean(); + if (!thisIsSelectedStrategy && (useStrategy == null ? false : useStrategy)) { + String failureFormat = "property %1$s cannot be used with %2$s"; + Collection validationFailureResults = new ArrayList(); + String message = String.format(failureFormat, strategyProperty.getDisplayName(), + primaryStrategy.getName()); + validationFailureResults.add(new ValidationResult.Builder() + .subject(strategyProperty.getDisplayName()) + .valid(false) + .explanation(message).build()); + return validationFailureResults; + } + return null; + } + +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.java new file mode 100644 index 000000000000..35b365ec7eb6 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialsStrategy; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; + +/** + * Partial implementation of CredentialsStrategy to support most simple property-based strategies. + */ +public abstract class AbstractCredentialsStrategy implements CredentialsStrategy { + private final String name; + private final PropertyDescriptor[] requiredProperties; + + public AbstractCredentialsStrategy(String name, PropertyDescriptor[] requiredProperties) { + this.name = name; + this.requiredProperties = requiredProperties; + } + + @Override + public boolean canCreatePrimaryCredential(Map properties) { + for (PropertyDescriptor requiredProperty : requiredProperties) { + boolean containsRequiredProperty = properties.containsKey(requiredProperty); + String propertyValue = properties.get(requiredProperty); + boolean containsValue = propertyValue != null; + if (!containsRequiredProperty || !containsValue) { + return false; + } + } + return true; + } + + @Override + public Collection validate(final ValidationContext validationContext, + final CredentialsStrategy primaryStrategy) { + boolean thisIsSelectedStrategy = this == primaryStrategy; + String requiredMessageFormat = "property %1$s must be set with %2$s"; + String excludedMessageFormat = "property %1$s cannot be used with %2$s"; + String failureFormat = thisIsSelectedStrategy ? requiredMessageFormat : excludedMessageFormat; + Collection validationFailureResults = null; + + for (PropertyDescriptor requiredProperty : requiredProperties) { + boolean requiredPropertyIsSet = validationContext.getProperty(requiredProperty).isSet(); + if (requiredPropertyIsSet != thisIsSelectedStrategy) { + String message = String.format(failureFormat, requiredProperty.getDisplayName(), + primaryStrategy.getName()); + if (validationFailureResults == null) { + validationFailureResults = new ArrayList<>(); + } + validationFailureResults.add(new ValidationResult.Builder() + .subject(requiredProperty.getDisplayName()) + .valid(false) + .explanation(message).build()); + } + } + + return validationFailureResults; + } + + @Override + public abstract GoogleCredentials getGoogleCredentials(Map properties) throws IOException; + + public String getName() { + return name; + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.java new file mode 100644 index 000000000000..3003c0d08e50 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/AbstractServiceAccountCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +/** + * Abstract class handling any of the service account related credential strategies, whether provided directly to NiFi + * or through a flat JSON file. + */ +public abstract class AbstractServiceAccountCredentialsStrategy extends AbstractCredentialsStrategy { + public AbstractServiceAccountCredentialsStrategy(String name, PropertyDescriptor[] requiredProperties) { + super(name, requiredProperties); + } + + protected abstract InputStream getServiceAccountJson(Map properties) throws IOException; + + @Override + public GoogleCredentials getGoogleCredentials(Map properties) throws IOException { + return GoogleCredentials.fromStream(getServiceAccountJson(properties)); + } + +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.java new file mode 100644 index 000000000000..5b5034b1f2a5 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ComputeEngineCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import com.google.auth.oauth2.ComputeEngineCredentials; +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors; + +import java.io.IOException; +import java.util.Map; + +/** + * Supports Google Compute Engine credentials. Fetches access tokens from the Google Compute Engine metadata server. + * + * @see + * Service Accounts for Instances + */ +public class ComputeEngineCredentialsStrategy extends AbstractBooleanCredentialsStrategy { + public ComputeEngineCredentialsStrategy() { + super("Compute Engine Credentials", CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS); + } + + @Override + public GoogleCredentials getGoogleCredentials(Map properties) throws IOException { + return new ComputeEngineCredentials(); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.java new file mode 100644 index 000000000000..55ebef780196 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ExplicitApplicationDefaultCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors; + +import java.io.IOException; +import java.util.Map; + + +/** + * Supports GCP Application Default Credentials. Compared to ImplicitApplicationDefaultCredentialsStrategy, this + * strategy is designed to be visible to the user, and depends on an affirmative selection from the user. + * + * @see + * Application Default Credentials + */ +public class ExplicitApplicationDefaultCredentialsStrategy extends AbstractBooleanCredentialsStrategy { + + public ExplicitApplicationDefaultCredentialsStrategy() { + super("Application Default Credentials", CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS); + } + + @Override + public GoogleCredentials getGoogleCredentials(Map properties) throws IOException { + return GoogleCredentials.getApplicationDefault(); + } + +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.java new file mode 100644 index 000000000000..95f97924130e --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/ImplicitApplicationDefaultCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.components.PropertyDescriptor; + +import java.io.IOException; +import java.util.Map; + +/** + * Supports Google Cloud Application Default Credentials. + * Compared to ExplicitApplicationDefaultCredentialsStrategy, this strategy is always + * willing to provide primary credentials, regardless of user input. It is intended to be used as an invisible + * fallback or default strategy. + */ +public class ImplicitApplicationDefaultCredentialsStrategy extends AbstractCredentialsStrategy { + + public ImplicitApplicationDefaultCredentialsStrategy() { + super("Application Default Credentials", new PropertyDescriptor[]{}); + } + + @Override + public GoogleCredentials getGoogleCredentials(Map properties) throws IOException { + return GoogleCredentials.getApplicationDefault(); + } + +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/JsonFileServiceAccountCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/JsonFileServiceAccountCredentialsStrategy.java new file mode 100644 index 000000000000..924a2ae22b40 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/JsonFileServiceAccountCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Map; + + +/** + * Supports service account credentials in a JSON file. + * + * @see + * Service Accounts + */ +public class JsonFileServiceAccountCredentialsStrategy extends AbstractServiceAccountCredentialsStrategy { + + public JsonFileServiceAccountCredentialsStrategy() { + super("Service Account Credentials (Json File)", new PropertyDescriptor[] { + CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE + }); + } + + @Override + protected InputStream getServiceAccountJson(Map properties) throws IOException { + String serviceAccountFile = properties.get(CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE); + return new BufferedInputStream(Files.newInputStream(Paths.get(serviceAccountFile))); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/JsonStringServiceAccountCredentialsStrategy.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/JsonStringServiceAccountCredentialsStrategy.java new file mode 100644 index 000000000000..3a40a7878fa0 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/factory/strategies/JsonStringServiceAccountCredentialsStrategy.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.processors.gcp.credentials.factory.strategies; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.Map; + + +/** + * Supports service account credentials provided as a JSON string. + * + * @see + * Service Accounts + */ +public class JsonStringServiceAccountCredentialsStrategy extends AbstractServiceAccountCredentialsStrategy { + + public JsonStringServiceAccountCredentialsStrategy() { + super("Service Account Credentials (Json String)", new PropertyDescriptor[] { + CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON + }); + } + + @Override + protected InputStream getServiceAccountJson(Map properties) { + String serviceAccountJson = properties.get(CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON); + return new ByteArrayInputStream(serviceAccountJson.getBytes()); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.java new file mode 100644 index 000000000000..5031687f959e --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsControllerService.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.processors.gcp.credentials.service; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processors.gcp.credentials.factory.CredentialsFactory; +import org.apache.nifi.reporting.InitializationException; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS; +/** + * Implementation of GCPCredentialsService interface + * + * @see GCPCredentialsService + */ +@CapabilityDescription("Defines credentials for Google Cloud Platform processors. " + + "Uses Application Default credentials without configuration. " + + "Application Default credentials support environmental variable (GOOGLE_APPLICATION_CREDENTIALS) pointing to " + + "a credential file, the config generated by `gcloud auth application-default login`, AppEngine/Compute Engine" + + " service accounts, etc.") +@Tags({ "gcp", "credentials","provider" }) +public class GCPCredentialsControllerService extends AbstractControllerService implements GCPCredentialsService { + + private static final List properties; + + static { + final List props = new ArrayList<>(); + props.add(USE_APPLICATION_DEFAULT_CREDENTIALS); + props.add(USE_COMPUTE_ENGINE_CREDENTIALS); + props.add(SERVICE_ACCOUNT_JSON_FILE); + props.add(SERVICE_ACCOUNT_JSON); + properties = Collections.unmodifiableList(props); + } + + private volatile GoogleCredentials googleCredentials; + protected final CredentialsFactory credentialsProviderFactory = new CredentialsFactory(); + + @Override + protected List getSupportedPropertyDescriptors() { + return properties; + } + + public GoogleCredentials getGoogleCredentials() throws ProcessException { + return googleCredentials; + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + return credentialsProviderFactory.validate(validationContext); + } + + @OnEnabled + public void onConfigured(final ConfigurationContext context) throws InitializationException { + try { + googleCredentials = credentialsProviderFactory.getGoogleCredentials(context.getProperties()); + } catch (IOException e) { + throw new InitializationException(e); + } + } + + @Override + public String toString() { + return "GCPCredentialsControllerService[id=" + getIdentifier() + "]"; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsService.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsService.java new file mode 100644 index 000000000000..9b22b23211c7 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsService.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.processors.gcp.credentials.service; + +import com.google.auth.oauth2.GoogleCredentials; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.processor.exception.ProcessException; + +/** + * GCPCredentialsService interface to support getting Google Cloud Platform + * GoogleCredentials used for instantiating Google cloud services. + * + * @see GoogleCredentials + */ +@Tags({"gcp", "security", "credentials", "auth", "session"}) +@CapabilityDescription("Provides GCP GoogleCredentials.") +public interface GCPCredentialsService extends ControllerService { + + /** + * Get Google Credentials + * @return Valid Google Credentials suitable for authorizing requests on the platform. + * @throws ProcessException process exception in case there is problem in getting credentials + */ + public GoogleCredentials getGoogleCredentials() throws ProcessException; +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.java new file mode 100644 index 000000000000..ff368b55df15 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/AbstractGCSProcessor.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.processors.gcp.storage; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.RetryParams; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; +import com.google.cloud.storage.spi.StorageRpc; +import com.google.common.collect.ImmutableList; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processors.gcp.AbstractGCPProcessor; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Base class for creating processors which connect to Google Cloud Storage. + * + * Every GCS processor operation requires a bucket, whether it's reading or writing from said bucket. + */ +public abstract class AbstractGCSProcessor extends AbstractGCPProcessor { + public static final Relationship REL_SUCCESS = + new Relationship.Builder().name("success") + .description("FlowFiles are routed to this relationship after a successful Google Cloud Storage operation.") + .build(); + public static final Relationship REL_FAILURE = + new Relationship.Builder().name("failure") + .description("FlowFiles are routed to this relationship if the Google Cloud Storage operation fails.") + .build(); + + public static final Set relationships = Collections.unmodifiableSet( + new HashSet<>(Arrays.asList(REL_SUCCESS, REL_FAILURE))); + + @Override + public Set getRelationships() { + return relationships; + } + + @Override + public List getSupportedPropertyDescriptors() { + return ImmutableList.builder() + .addAll(super.getSupportedPropertyDescriptors()) + .build(); + } + + @Override + protected StorageOptions getServiceOptions(ProcessContext context, GoogleCredentials credentials) { + final String projectId = context.getProperty(PROJECT_ID).getValue(); + final Integer retryCount = Integer.valueOf(context.getProperty(RETRY_COUNT).getValue()); + + return StorageOptions.newBuilder() + .setCredentials(credentials) + .setProjectId(projectId) + .setRetryParams(RetryParams.newBuilder() + .setRetryMaxAttempts(retryCount) + .setRetryMinAttempts(retryCount) + .build()) + .build(); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java new file mode 100644 index 000000000000..80bb9c0a5896 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObject.java @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.Storage; +import com.google.common.collect.ImmutableList; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.SupportsBatching; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_DESC; + + +@SupportsBatching +@Tags({"google cloud", "gcs", "google", "storage", "delete"}) +@CapabilityDescription("Deletes objects from a Google Cloud Bucket. " + + "If attempting to delete a file that does not exist, FlowFile is routed to success.") +@SeeAlso({PutGCSObject.class, FetchGCSObject.class, ListGCSBucket.class}) +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +public class DeleteGCSObject extends AbstractGCSProcessor { + public static final PropertyDescriptor BUCKET = new PropertyDescriptor + .Builder().name("gcs-bucket") + .displayName("Bucket") + .description(BUCKET_DESC) + .required(true) + .defaultValue("${" + BUCKET_ATTR + "}") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor KEY = new PropertyDescriptor + .Builder().name("gcs-key") + .displayName("Key") + .description(KEY_DESC) + .required(true) + .defaultValue("${filename}") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor GENERATION = new PropertyDescriptor.Builder() + .name("gcs-generation") + .displayName("Generation") + .description("The generation of the object to be deleted. If null, will use latest version of the object.") + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .expressionLanguageSupported(true) + .required(false) + .build(); + + @Override + public List getSupportedPropertyDescriptors() { + return ImmutableList.builder() + .addAll(super.getSupportedPropertyDescriptors()) + .add(BUCKET) + .add(KEY) + .add(GENERATION) + .build(); + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + final long startNanos = System.nanoTime(); + + final String bucket = context.getProperty(BUCKET) + .evaluateAttributeExpressions(flowFile) + .getValue(); + final String key = context.getProperty(KEY) + .evaluateAttributeExpressions(flowFile) + .getValue(); + + final Long generation = context.getProperty(GENERATION) + .evaluateAttributeExpressions(flowFile) + .asLong(); + + + final Storage storage = getCloudService(); + + // Deletes a key on Google Cloud + try { + storage.delete(BlobId.of(bucket, key, generation)); + } catch (Exception e) { + getLogger().error(e.getMessage(), e); + flowFile = session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + return; + } + + session.transfer(flowFile, REL_SUCCESS); + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + getLogger().info("Successfully deleted GCS Object for {} in {} millis; routing to success", new Object[]{flowFile, millis}); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java new file mode 100644 index 000000000000..a65158a13e43 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/FetchGCSObject.java @@ -0,0 +1,346 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.ReadChannel; +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.common.collect.ImmutableList; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.nio.channels.Channels; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_DESC; + +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@Tags({"google cloud", "google", "storage", "gcs", "fetch"}) +@CapabilityDescription("Fetches a file from a Google Cloud Bucket. Designed to be used in tandem with ListGCSBucket.") +@SeeAlso({ListGCSBucket.class, PutGCSObject.class, DeleteGCSObject.class}) +@WritesAttributes({ + @WritesAttribute(attribute = "filename", description = "The name of the file, parsed if possible from the " + + "Content-Disposition response header"), + @WritesAttribute(attribute = BUCKET_ATTR, description = BUCKET_DESC), + @WritesAttribute(attribute = KEY_ATTR, description = KEY_DESC), + @WritesAttribute(attribute = SIZE_ATTR, description = SIZE_DESC), + @WritesAttribute(attribute = CACHE_CONTROL_ATTR, description = CACHE_CONTROL_DESC), + @WritesAttribute(attribute = COMPONENT_COUNT_ATTR, description = COMPONENT_COUNT_DESC), + @WritesAttribute(attribute = CONTENT_DISPOSITION_ATTR, description = CONTENT_DISPOSITION_DESC), + @WritesAttribute(attribute = CONTENT_ENCODING_ATTR, description = CONTENT_ENCODING_DESC), + @WritesAttribute(attribute = CONTENT_LANGUAGE_ATTR, description = CONTENT_LANGUAGE_DESC), + @WritesAttribute(attribute = "mime.type", description = "The MIME/Content-Type of the object"), + @WritesAttribute(attribute = CRC32C_ATTR, description = CRC32C_DESC), + @WritesAttribute(attribute = CREATE_TIME_ATTR, description = CREATE_TIME_DESC), + @WritesAttribute(attribute = UPDATE_TIME_ATTR, description = UPDATE_TIME_DESC), + @WritesAttribute(attribute = ENCRYPTION_ALGORITHM_ATTR, description = ENCRYPTION_ALGORITHM_DESC), + @WritesAttribute(attribute = ENCRYPTION_SHA256_ATTR, description = ENCRYPTION_SHA256_DESC), + @WritesAttribute(attribute = ETAG_ATTR, description = ETAG_DESC), + @WritesAttribute(attribute = GENERATED_ID_ATTR, description = GENERATED_ID_DESC), + @WritesAttribute(attribute = GENERATION_ATTR, description = GENERATION_DESC), + @WritesAttribute(attribute = MD5_ATTR, description = MD5_DESC), + @WritesAttribute(attribute = MEDIA_LINK_ATTR, description = MEDIA_LINK_DESC), + @WritesAttribute(attribute = METAGENERATION_ATTR, description = METAGENERATION_DESC), + @WritesAttribute(attribute = OWNER_ATTR, description = OWNER_DESC), + @WritesAttribute(attribute = OWNER_TYPE_ATTR, description = OWNER_TYPE_DESC), + @WritesAttribute(attribute = URI_ATTR, description = URI_DESC) +}) +public class FetchGCSObject extends AbstractGCSProcessor { + public static final PropertyDescriptor BUCKET = new PropertyDescriptor + .Builder().name("gcs-bucket") + .displayName("Bucket") + .description(BUCKET_DESC) + .required(true) + .defaultValue("${" + BUCKET_ATTR + "}") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor KEY = new PropertyDescriptor + .Builder().name("gcs-key") + .displayName("Key") + .description(KEY_DESC) + .required(true) + .defaultValue("${" + CoreAttributes.FILENAME.key() + "}") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor GENERATION = new PropertyDescriptor.Builder() + .name("gcs-generation") + .displayName("Object Generation") + .description("The generation of the Object to download. If null, will download latest generation.") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.POSITIVE_LONG_VALIDATOR) + .required(false) + .build(); + + public static final PropertyDescriptor ENCRYPTION_KEY = new PropertyDescriptor.Builder() + .name("gcs-server-side-encryption-key") + .displayName("Server Side Encryption Key") + .description("An AES256 Key (encoded in base64) which the object has been encrypted in.") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(true) + .build(); + + @Override + public List getSupportedPropertyDescriptors() { + return ImmutableList.builder() + .addAll(super.getSupportedPropertyDescriptors()) + .add(BUCKET) + .add(KEY) + .add(GENERATION) + .add(ENCRYPTION_KEY) + .build(); + } + + + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + final long startNanos = System.nanoTime(); + + String bucketName = context.getProperty(BUCKET) + .evaluateAttributeExpressions(flowFile) + .getValue(); + String key = context.getProperty(KEY) + .evaluateAttributeExpressions(flowFile) + .getValue(); + Long generation = context.getProperty(GENERATION) + .evaluateAttributeExpressions(flowFile) + .asLong(); + String encryptionKey = context.getProperty(ENCRYPTION_KEY) + .evaluateAttributeExpressions(flowFile) + .getValue(); + + final Storage storage = getCloudService(); + final Map attributes = new HashMap<>(); + final BlobId blobId = BlobId.of(bucketName, key, generation); + + try { + final List blobSourceOptions = new ArrayList<>(2); + + if (encryptionKey != null) { + blobSourceOptions.add(Storage.BlobSourceOption.decryptionKey(encryptionKey)); + } + + if (generation != null) { + blobSourceOptions.add(Storage.BlobSourceOption.generationMatch()); + } + + final Blob blob = storage.get(blobId); + + if (blob == null) { + throw new StorageException(404, "Blob " + blobId + " not found"); + } + + final ReadChannel reader = storage.reader(blobId, blobSourceOptions.toArray(new Storage.BlobSourceOption[blobSourceOptions.size()])); + + flowFile = session.importFrom(Channels.newInputStream(reader), flowFile); + + attributes.put(BUCKET_ATTR, blob.getBucket()); + attributes.put(KEY_ATTR, blob.getName()); + + if (blob.getSize() != null) { + attributes.put(SIZE_ATTR, String.valueOf(blob.getSize())); + } + + if (blob.getCacheControl() != null) { + attributes.put(CACHE_CONTROL_ATTR, blob.getCacheControl()); + } + + if (blob.getComponentCount() != null) { + attributes.put(COMPONENT_COUNT_ATTR, String.valueOf(blob.getComponentCount())); + } + + if (blob.getContentEncoding() != null) { + attributes.put(CONTENT_ENCODING_ATTR, blob.getContentEncoding()); + } + + if (blob.getContentLanguage() != null) { + attributes.put(CONTENT_LANGUAGE_ATTR, blob.getContentLanguage()); + } + + if (blob.getContentType() != null) { + attributes.put(CoreAttributes.MIME_TYPE.key(), blob.getContentType()); + } + + if (blob.getCrc32c() != null) { + attributes.put(CRC32C_ATTR, blob.getCrc32c()); + } + + if (blob.getCustomerEncryption() != null) { + final BlobInfo.CustomerEncryption encryption = blob.getCustomerEncryption(); + + attributes.put(ENCRYPTION_ALGORITHM_ATTR, encryption.getEncryptionAlgorithm()); + attributes.put(ENCRYPTION_SHA256_ATTR, encryption.getKeySha256()); + } + + if (blob.getEtag() != null) { + attributes.put(ETAG_ATTR, blob.getEtag()); + } + + if (blob.getGeneratedId() != null) { + attributes.put(GENERATED_ID_ATTR, blob.getGeneratedId()); + } + + if (blob.getGeneration() != null) { + attributes.put(GENERATION_ATTR, String.valueOf(blob.getGeneration())); + } + + if (blob.getMd5() != null) { + attributes.put(MD5_ATTR, blob.getMd5()); + } + + if (blob.getMediaLink() != null) { + attributes.put(MEDIA_LINK_ATTR, blob.getMediaLink()); + } + + if (blob.getMetageneration() != null) { + attributes.put(METAGENERATION_ATTR, String.valueOf(blob.getMetageneration())); + } + + if (blob.getOwner() != null) { + final Acl.Entity entity = blob.getOwner(); + + if (entity instanceof Acl.User) { + attributes.put(OWNER_ATTR, ((Acl.User) entity).getEmail()); + attributes.put(OWNER_TYPE_ATTR, "user"); + } else if (entity instanceof Acl.Group) { + attributes.put(OWNER_ATTR, ((Acl.Group) entity).getEmail()); + attributes.put(OWNER_TYPE_ATTR, "group"); + } else if (entity instanceof Acl.Domain) { + attributes.put(OWNER_ATTR, ((Acl.Domain) entity).getDomain()); + attributes.put(OWNER_TYPE_ATTR, "domain"); + } else if (entity instanceof Acl.Project) { + attributes.put(OWNER_ATTR, ((Acl.Project) entity).getProjectId()); + attributes.put(OWNER_TYPE_ATTR, "project"); + } + } + + if (blob.getSelfLink() != null) { + attributes.put(URI_ATTR, blob.getSelfLink()); + } + + if (blob.getContentDisposition() != null) { + attributes.put(CONTENT_DISPOSITION_ATTR, blob.getContentDisposition()); + + final Util.ParsedContentDisposition parsedContentDisposition = Util.parseContentDisposition(blob.getContentDisposition()); + + if (parsedContentDisposition != null) { + attributes.put(CoreAttributes.FILENAME.key(), parsedContentDisposition.getFileName()); + } + } + + if (blob.getCreateTime() != null) { + attributes.put(CREATE_TIME_ATTR, String.valueOf(blob.getCreateTime())); + } + + if (blob.getUpdateTime() != null) { + attributes.put(UPDATE_TIME_ATTR, String.valueOf(blob.getUpdateTime())); + } + + } catch (StorageException e) { + getLogger().error(e.getMessage(), e); + flowFile = session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + return; + } + + if (!attributes.isEmpty()) { + flowFile = session.putAllAttributes(flowFile, attributes); + } + session.transfer(flowFile, REL_SUCCESS); + + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + getLogger().info("Successfully retrieved GCS Object for {} in {} millis; routing to success", new Object[]{flowFile, millis}); + session.getProvenanceReporter().fetch( + flowFile, + "https://" + bucketName + ".storage.googleapis.com/" + key, + millis); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java new file mode 100644 index 000000000000..899895372801 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/ListGCSBucket.java @@ -0,0 +1,409 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.Page; +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.common.collect.ImmutableList; +import org.apache.nifi.annotation.behavior.Stateful; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.Relationship; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_DESC; + +/** + * List objects in a google cloud storage bucket by object name pattern. + */ +@Tags({"google cloud", "google", "storage", "gcs", "list"}) +@CapabilityDescription("Retrieves a listing of objects from an GCS bucket. For each object that is listed, creates a FlowFile that represents " + + "the object so that it can be fetched in conjunction with FetchGCSObject. This Processor is designed to run on Primary Node only " + + "in a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left off without duplicating " + + "all of the data.") +@Stateful(scopes = Scope.CLUSTER, description = "After performing a listing of keys, the timestamp of the newest key is stored, " + + "along with the keys that share that same timestamp. This allows the Processor to list only keys that have been added or modified after " + + "this date the next time that the Processor is run. State is stored across the cluster so that this Processor can be run on Primary Node only and if a new Primary " + + "Node is selected, the new node can pick up where the previous node left off, without duplicating the data.") +@SeeAlso({PutGCSObject.class, DeleteGCSObject.class, FetchGCSObject.class}) +@WritesAttributes({ + @WritesAttribute(attribute = "filename", description = "The name of the file"), + @WritesAttribute(attribute = BUCKET_ATTR, description = BUCKET_DESC), + @WritesAttribute(attribute = KEY_ATTR, description = KEY_DESC), + @WritesAttribute(attribute = SIZE_ATTR, description = SIZE_DESC), + @WritesAttribute(attribute = CACHE_CONTROL_ATTR, description = CACHE_CONTROL_DESC), + @WritesAttribute(attribute = COMPONENT_COUNT_ATTR, description = COMPONENT_COUNT_DESC), + @WritesAttribute(attribute = CONTENT_DISPOSITION_ATTR, description = CONTENT_DISPOSITION_DESC), + @WritesAttribute(attribute = CONTENT_ENCODING_ATTR, description = CONTENT_ENCODING_DESC), + @WritesAttribute(attribute = CONTENT_LANGUAGE_ATTR, description = CONTENT_LANGUAGE_DESC), + @WritesAttribute(attribute = "mime.type", description = "The MIME/Content-Type of the object"), + @WritesAttribute(attribute = CRC32C_ATTR, description = CRC32C_DESC), + @WritesAttribute(attribute = CREATE_TIME_ATTR, description = CREATE_TIME_DESC), + @WritesAttribute(attribute = UPDATE_TIME_ATTR, description = UPDATE_TIME_DESC), + @WritesAttribute(attribute = ENCRYPTION_ALGORITHM_ATTR, description = ENCRYPTION_ALGORITHM_DESC), + @WritesAttribute(attribute = ENCRYPTION_SHA256_ATTR, description = ENCRYPTION_SHA256_DESC), + @WritesAttribute(attribute = ETAG_ATTR, description = ETAG_DESC), + @WritesAttribute(attribute = GENERATED_ID_ATTR, description = GENERATED_ID_DESC), + @WritesAttribute(attribute = GENERATION_ATTR, description = GENERATION_DESC), + @WritesAttribute(attribute = MD5_ATTR, description = MD5_DESC), + @WritesAttribute(attribute = MEDIA_LINK_ATTR, description = MEDIA_LINK_DESC), + @WritesAttribute(attribute = METAGENERATION_ATTR, description = METAGENERATION_DESC), + @WritesAttribute(attribute = OWNER_ATTR, description = OWNER_DESC), + @WritesAttribute(attribute = OWNER_TYPE_ATTR, description = OWNER_TYPE_DESC), + @WritesAttribute(attribute = URI_ATTR, description = URI_DESC) +}) +public class ListGCSBucket extends AbstractGCSProcessor { + public static final PropertyDescriptor BUCKET = new PropertyDescriptor + .Builder().name("gcs-bucket") + .displayName("Bucket") + .description(BUCKET_DESC) + .required(true) + .expressionLanguageSupported(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor PREFIX = new PropertyDescriptor.Builder() + .name("gcs-prefix") + .displayName("Prefix") + .description("The prefix used to filter the object list. In most cases, it should end with a forward slash ('/').") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor USE_GENERATIONS = new PropertyDescriptor.Builder() + .name("gcs-use-generations") + .displayName("Use Generations") + .expressionLanguageSupported(false) + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .allowableValues("true", "false") + .defaultValue("false") + .description("Specifies whether to use GCS Generations, if applicable. If false, only the latest version of each object will be returned.") + .build(); + + @Override + public List getSupportedPropertyDescriptors() { + return ImmutableList.builder() + .addAll(super.getSupportedPropertyDescriptors()) + .add(BUCKET) + .add(PREFIX) + .add(USE_GENERATIONS) + .build(); + } + + public static final Set relationships = Collections.unmodifiableSet( + new HashSet<>(Collections.singletonList(REL_SUCCESS))); + @Override + public Set getRelationships() { + return relationships; + } + + // State tracking + public static final String CURRENT_TIMESTAMP = "currentTimestamp"; + public static final String CURRENT_KEY_PREFIX = "key-"; + protected long currentTimestamp = 0L; + protected Set currentKeys; + + + private Set extractKeys(final StateMap stateMap) { + return stateMap.toMap().entrySet().parallelStream() + .filter(x -> x.getKey().startsWith(CURRENT_KEY_PREFIX)) + .map(Map.Entry::getValue) + .collect(Collectors.toSet()); + } + + void restoreState(final ProcessContext context) throws IOException { + final StateMap stateMap = context.getStateManager().getState(Scope.CLUSTER); + if (stateMap.getVersion() == -1L || stateMap.get(CURRENT_TIMESTAMP) == null || stateMap.get(CURRENT_KEY_PREFIX+"0") == null) { + currentTimestamp = 0L; + currentKeys = new HashSet<>(); + } else { + currentTimestamp = Long.parseLong(stateMap.get(CURRENT_TIMESTAMP)); + currentKeys = extractKeys(stateMap); + } + } + + void persistState(final ProcessContext context) { + Map state = new HashMap<>(); + state.put(CURRENT_TIMESTAMP, String.valueOf(currentTimestamp)); + int i = 0; + for (String key : currentKeys) { + state.put(CURRENT_KEY_PREFIX+i, key); + i++; + } + try { + context.getStateManager().setState(state, Scope.CLUSTER); + } catch (IOException ioe) { + getLogger().error("Failed to save cluster-wide state. If NiFi is restarted, data duplication may occur", ioe); + } + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + try { + restoreState(context); + } catch (IOException e) { + getLogger().error("Failed to restore processor state; yielding", e); + context.yield(); + return; + } + + final long startNanos = System.nanoTime(); + + final String bucket = context.getProperty(BUCKET).getValue(); + + final String prefix = context.getProperty(PREFIX).getValue(); + + final boolean useGenerations = context.getProperty(USE_GENERATIONS).asBoolean(); + + List listOptions = new ArrayList<>(); + if (prefix != null) { + listOptions.add(Storage.BlobListOption.prefix(prefix)); + } + + if (useGenerations) { + listOptions.add(Storage.BlobListOption.versions(true)); + } + + final Storage storage = getCloudService(); + int listCount = 0; + long maxTimestamp = 0L; + + + Page blobPages = storage.list(bucket, listOptions.toArray(new Storage.BlobListOption[listOptions.size()])); + do { + for (Blob blob : blobPages.getValues()) { + long lastModified = blob.getUpdateTime(); + if (lastModified < currentTimestamp + || lastModified == currentTimestamp && currentKeys.contains(blob.getName())) { + continue; + } + + // Create attributes + final Map attributes = new HashMap<>(); + + attributes.put(BUCKET_ATTR, blob.getBucket()); + attributes.put(KEY_ATTR, blob.getName()); + + if (blob.getSize() != null) { + attributes.put(SIZE_ATTR, String.valueOf(blob.getSize())); + } + + if (blob.getCacheControl() != null) { + attributes.put(CACHE_CONTROL_ATTR, blob.getCacheControl()); + } + + if (blob.getComponentCount() != null) { + attributes.put(COMPONENT_COUNT_ATTR, String.valueOf(blob.getComponentCount())); + } + + if (blob.getContentDisposition() != null) { + attributes.put(CONTENT_DISPOSITION_ATTR, blob.getContentDisposition()); + } + + if (blob.getContentEncoding() != null) { + attributes.put(CONTENT_ENCODING_ATTR, blob.getContentEncoding()); + } + + if (blob.getContentLanguage() != null) { + attributes.put(CONTENT_LANGUAGE_ATTR, blob.getContentLanguage()); + } + + if (blob.getContentType() != null) { + attributes.put(CoreAttributes.MIME_TYPE.key(), blob.getContentType()); + } + + if (blob.getCrc32c() != null) { + attributes.put(CRC32C_ATTR, blob.getCrc32c()); + } + + if (blob.getCustomerEncryption() != null) { + final BlobInfo.CustomerEncryption encryption = blob.getCustomerEncryption(); + + attributes.put(ENCRYPTION_ALGORITHM_ATTR, encryption.getEncryptionAlgorithm()); + attributes.put(ENCRYPTION_SHA256_ATTR, encryption.getKeySha256()); + } + + if (blob.getEtag() != null) { + attributes.put(ETAG_ATTR, blob.getEtag()); + } + + if (blob.getGeneratedId() != null) { + attributes.put(GENERATED_ID_ATTR, blob.getGeneratedId()); + } + + if (blob.getGeneration() != null) { + attributes.put(GENERATION_ATTR, String.valueOf(blob.getGeneration())); + } + + if (blob.getMd5() != null) { + attributes.put(MD5_ATTR, blob.getMd5()); + } + + if (blob.getMediaLink() != null) { + attributes.put(MEDIA_LINK_ATTR, blob.getMediaLink()); + } + + if (blob.getMetageneration() != null) { + attributes.put(METAGENERATION_ATTR, String.valueOf(blob.getMetageneration())); + } + + if (blob.getOwner() != null) { + final Acl.Entity entity = blob.getOwner(); + + if (entity instanceof Acl.User) { + attributes.put(OWNER_ATTR, ((Acl.User) entity).getEmail()); + attributes.put(OWNER_TYPE_ATTR, "user"); + } else if (entity instanceof Acl.Group) { + attributes.put(OWNER_ATTR, ((Acl.Group) entity).getEmail()); + attributes.put(OWNER_TYPE_ATTR, "group"); + } else if (entity instanceof Acl.Domain) { + attributes.put(OWNER_ATTR, ((Acl.Domain) entity).getDomain()); + attributes.put(OWNER_TYPE_ATTR, "domain"); + } else if (entity instanceof Acl.Project) { + attributes.put(OWNER_ATTR, ((Acl.Project) entity).getProjectId()); + attributes.put(OWNER_TYPE_ATTR, "project"); + } + } + + if (blob.getSelfLink() != null) { + attributes.put(URI_ATTR, blob.getSelfLink()); + } + + attributes.put(CoreAttributes.FILENAME.key(), blob.getName()); + + if (blob.getCreateTime() != null) { + attributes.put(CREATE_TIME_ATTR, String.valueOf(blob.getCreateTime())); + } + + if (blob.getUpdateTime() != null) { + attributes.put(UPDATE_TIME_ATTR, String.valueOf(blob.getUpdateTime())); + } + + // Create the flowfile + FlowFile flowFile = session.create(); + flowFile = session.putAllAttributes(flowFile, attributes); + session.transfer(flowFile, REL_SUCCESS); + + // Update state + if (lastModified > maxTimestamp) { + maxTimestamp = lastModified; + currentKeys.clear(); + } + if (lastModified == maxTimestamp) { + currentKeys.add(blob.getName()); + } + listCount++; + } + + blobPages = blobPages.getNextPage(); + commit(context, session, listCount); + listCount = 0; + } while (blobPages != null); + + currentTimestamp = maxTimestamp; + + final long listMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + getLogger().info("Successfully listed GCS bucket {} in {} millis", new Object[]{bucket, listMillis}); + + if (!commit(context, session, listCount)) { + if (currentTimestamp > 0) { + persistState(context); + } + getLogger().debug("No new objects in GCS bucket {} to list. Yielding.", new Object[]{bucket}); + context.yield(); + } + } + + private boolean commit(final ProcessContext context, final ProcessSession session, int listCount) { + boolean willCommit = listCount > 0; + if (willCommit) { + getLogger().info("Successfully listed {} new files from GCS; routing to success", new Object[] {listCount}); + session.commit(); + persistState(context); + } + return willCommit; + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java new file mode 100644 index 000000000000..86f906c9a045 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/PutGCSObject.java @@ -0,0 +1,538 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.common.collect.ImmutableList; +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttributes; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import java.io.BufferedInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static com.google.cloud.storage.Storage.PredefinedAcl.ALL_AUTHENTICATED_USERS; +import static com.google.cloud.storage.Storage.PredefinedAcl.AUTHENTICATED_READ; +import static com.google.cloud.storage.Storage.PredefinedAcl.BUCKET_OWNER_FULL_CONTROL; +import static com.google.cloud.storage.Storage.PredefinedAcl.BUCKET_OWNER_READ; +import static com.google.cloud.storage.Storage.PredefinedAcl.PRIVATE; +import static com.google.cloud.storage.Storage.PredefinedAcl.PROJECT_PRIVATE; +import static com.google.cloud.storage.Storage.PredefinedAcl.PUBLIC_READ; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_DESC; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_DESC; + + +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED) +@Tags({"google", "google cloud", "gcs", "archive", "put"}) +@CapabilityDescription("Puts flow files to a Google Cloud Bucket.") +@SeeAlso({FetchGCSObject.class, DeleteGCSObject.class, ListGCSBucket.class}) +@DynamicProperty(name = "The name of a User-Defined Metadata field to add to the GCS Object", + value = "The value of a User-Defined Metadata field to add to the GCS Object", + description = "Allows user-defined metadata to be added to the GCS object as key/value pairs", + supportsExpressionLanguage = true) +@ReadsAttributes({ + @ReadsAttribute(attribute = "filename", description = "Uses the FlowFile's filename as the filename for the " + + "GCS object"), + @ReadsAttribute(attribute = "mime.type", description = "Uses the FlowFile's MIME type as the content-type for " + + "the GCS object") +}) +@WritesAttributes({ + @WritesAttribute(attribute = BUCKET_ATTR, description = BUCKET_DESC), + @WritesAttribute(attribute = KEY_ATTR, description = KEY_DESC), + @WritesAttribute(attribute = SIZE_ATTR, description = SIZE_DESC), + @WritesAttribute(attribute = CACHE_CONTROL_ATTR, description = CACHE_CONTROL_DESC), + @WritesAttribute(attribute = COMPONENT_COUNT_ATTR, description = COMPONENT_COUNT_DESC), + @WritesAttribute(attribute = CONTENT_DISPOSITION_ATTR, description = CONTENT_DISPOSITION_DESC), + @WritesAttribute(attribute = CONTENT_ENCODING_ATTR, description = CONTENT_ENCODING_DESC), + @WritesAttribute(attribute = CONTENT_LANGUAGE_ATTR, description = CONTENT_LANGUAGE_DESC), + @WritesAttribute(attribute = "mime.type", description = "The MIME/Content-Type of the object"), + @WritesAttribute(attribute = CRC32C_ATTR, description = CRC32C_DESC), + @WritesAttribute(attribute = CREATE_TIME_ATTR, description = CREATE_TIME_DESC), + @WritesAttribute(attribute = UPDATE_TIME_ATTR, description = UPDATE_TIME_DESC), + @WritesAttribute(attribute = ENCRYPTION_ALGORITHM_ATTR, description = ENCRYPTION_ALGORITHM_DESC), + @WritesAttribute(attribute = ENCRYPTION_SHA256_ATTR, description = ENCRYPTION_SHA256_DESC), + @WritesAttribute(attribute = ETAG_ATTR, description = ETAG_DESC), + @WritesAttribute(attribute = GENERATED_ID_ATTR, description = GENERATED_ID_DESC), + @WritesAttribute(attribute = GENERATION_ATTR, description = GENERATION_DESC), + @WritesAttribute(attribute = MD5_ATTR, description = MD5_DESC), + @WritesAttribute(attribute = MEDIA_LINK_ATTR, description = MEDIA_LINK_DESC), + @WritesAttribute(attribute = METAGENERATION_ATTR, description = METAGENERATION_DESC), + @WritesAttribute(attribute = OWNER_ATTR, description = OWNER_DESC), + @WritesAttribute(attribute = OWNER_TYPE_ATTR, description = OWNER_TYPE_DESC), + @WritesAttribute(attribute = URI_ATTR, description = URI_DESC) +}) +public class PutGCSObject extends AbstractGCSProcessor { + public static final PropertyDescriptor BUCKET = new PropertyDescriptor + .Builder().name("gcs-bucket") + .displayName("Bucket") + .description(BUCKET_DESC) + .required(true) + .defaultValue("${" + BUCKET_ATTR + "}") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor KEY = new PropertyDescriptor + .Builder().name("gcs-key") + .displayName("Key") + .description(KEY_DESC) + .required(true) + .defaultValue("${filename}") + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor CONTENT_TYPE = new PropertyDescriptor + .Builder().name("gcs-content-type") + .displayName("Content Type") + .description("Content Type for the file, i.e. text/plain") + .defaultValue("${mime.type}") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final PropertyDescriptor MD5 = new PropertyDescriptor + .Builder().name("gcs-object-md5") + .displayName("MD5 Hash") + .description("MD5 Hash (encoded in Base64) of the file for server-side validation.") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + + public static final PropertyDescriptor CRC32C = new PropertyDescriptor + .Builder().name("gcs-object-crc32c") + .displayName("CRC32C Checksum") + .description("CRC32C Checksum (encoded in Base64, big-Endian order) of the file for server-side validation.") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .build(); + + public static final AllowableValue ACL_ALL_AUTHENTICATED_USERS = new AllowableValue( + ALL_AUTHENTICATED_USERS.name(), "All Authenticated Users", "Gives the bucket or object owner OWNER " + + "permission, and gives all authenticated Google account holders READER and WRITER permissions. " + + "All other permissions are removed." + ); + + public static final AllowableValue ACL_AUTHENTICATED_READ = new AllowableValue( + AUTHENTICATED_READ.name(), "Authenticated Read", "Gives the bucket or object owner OWNER permission, " + + "and gives all authenticated Google account holders READER permission. All other permissions are removed." + ); + + public static final AllowableValue ACL_BUCKET_OWNER_FULL_CONTROL = new AllowableValue( + BUCKET_OWNER_FULL_CONTROL.name(), "Bucket Owner Full Control", "Gives the object and bucket owners OWNER " + + "permission. All other permissions are removed." + ); + + public static final AllowableValue ACL_BUCKET_OWNER_READ = new AllowableValue( + BUCKET_OWNER_READ.name(), "Bucket Owner Read Only", "Gives the object owner OWNER permission, and gives " + + "the bucket owner READER permission. All other permissions are removed." + ); + + public static final AllowableValue ACL_PRIVATE = new AllowableValue( + PRIVATE.name(), "Private", "Gives the bucket or object owner OWNER permission for a bucket or object, " + + "and removes all other access permissions." + ); + + public static final AllowableValue ACL_PROJECT_PRIVATE = new AllowableValue( + PROJECT_PRIVATE.name(), "Project Private", "Gives permission to the project team based on their roles. " + + "Anyone who is part of the team has READER permission. Project owners and project editors have OWNER " + + "permission. This is the default ACL for newly created buckets. This is also the default ACL for newly " + + "created objects unless the default object ACL for that bucket has been changed." + ); + + public static final AllowableValue ACL_PUBLIC_READ = new AllowableValue( + PUBLIC_READ.name(), "Public Read Only", "Gives the bucket or object owner OWNER permission, and gives all " + + "users, both authenticated and anonymous, READER permission. When you apply this to an object, anyone on " + + "the Internet can read the object without authenticating." + ); + + public static final PropertyDescriptor ACL = new PropertyDescriptor.Builder() + .name("gcs-object-acl") + .displayName("Object ACL") + .description("Access Control to be attached to the object uploaded. Not providing this will revert to bucket defaults.") + .required(false) + .allowableValues( + ACL_ALL_AUTHENTICATED_USERS, + ACL_AUTHENTICATED_READ, + ACL_BUCKET_OWNER_FULL_CONTROL, + ACL_BUCKET_OWNER_READ, + ACL_PRIVATE, + ACL_PROJECT_PRIVATE, + ACL_PUBLIC_READ) + .build(); + + public static final PropertyDescriptor ENCRYPTION_KEY = new PropertyDescriptor.Builder() + .name("gcs-server-side-encryption-key") + .displayName("Server Side Encryption Key") + .description("An AES256 Encryption Key (encoded in base64) for server-side encryption of the object.") + .required(false) + .expressionLanguageSupported(true) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .sensitive(true) + .build(); + + + public static final PropertyDescriptor OVERWRITE = new PropertyDescriptor.Builder() + .name("gcs-overwrite-object") + .displayName("Overwrite Object") + .description("If false, the upload to GCS will succeed only if the object does not exist.") + .required(true) + .addValidator(StandardValidators.BOOLEAN_VALIDATOR) + .allowableValues("true", "false") + .defaultValue("true") + .build(); + + public static final AllowableValue CD_INLINE = new AllowableValue( + "inline", "Inline", "Indicates that the object should be loaded and rendered within the browser." + ); + + public static final AllowableValue CD_ATTACHMENT = new AllowableValue( + "attachment", "Attachment", "Indicates that the object should be saved (using a Save As... dialog) rather " + + "than opened directly within the browser" + ); + + public static final PropertyDescriptor CONTENT_DISPOSITION_TYPE = new PropertyDescriptor.Builder() + .name("gcs-content-disposition-type") + .displayName("Content Disposition Type") + .description("Type of RFC-6266 Content Disposition to be attached to the object") + .required(false) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .allowableValues(CD_INLINE, CD_ATTACHMENT) + .build(); + + @Override + public List getSupportedPropertyDescriptors() { + return ImmutableList.builder() + .addAll(super.getSupportedPropertyDescriptors()) + .add(BUCKET) + .add(KEY) + .add(CONTENT_TYPE) + .add(MD5) + .add(CRC32C) + .add(ACL) + .add(ENCRYPTION_KEY) + .add(OVERWRITE) + .add(CONTENT_DISPOSITION_TYPE) + .build(); + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .dynamic(true) + .build(); + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException { + FlowFile flowFile = session.get(); + if (flowFile == null) { + return; + } + + final long startNanos = System.nanoTime(); + + final String bucket = context.getProperty(BUCKET) + .evaluateAttributeExpressions(flowFile) + .getValue(); + final String key = context.getProperty(KEY) + .evaluateAttributeExpressions(flowFile) + .getValue(); + final boolean overwrite = context.getProperty(OVERWRITE).asBoolean(); + + final FlowFile ff = flowFile; + final String ffFilename = ff.getAttributes().get(CoreAttributes.FILENAME.key()); + final Map attributes = new HashMap<>(); + + try { + final Storage storage = getCloudService(); + session.read(flowFile, new InputStreamCallback() { + @Override + public void process(InputStream rawIn) throws IOException { + try (final InputStream in = new BufferedInputStream(rawIn)) { + final BlobId id = BlobId.of(bucket, key); + final BlobInfo.Builder blobInfoBuilder = BlobInfo.newBuilder(id); + final List blobWriteOptions = new ArrayList<>(); + + if (!overwrite) { + blobWriteOptions.add(Storage.BlobWriteOption.doesNotExist()); + } + + final String contentDispositionType = context.getProperty(CONTENT_DISPOSITION_TYPE).getValue(); + if (contentDispositionType != null) { + blobInfoBuilder.setContentDisposition(contentDispositionType + "; filename=" + ffFilename); + } + + final String contentType = context.getProperty(CONTENT_TYPE) + .evaluateAttributeExpressions(ff).getValue(); + if (contentType != null) { + blobInfoBuilder.setContentType(contentType); + } + + final String md5 = context.getProperty(MD5) + .evaluateAttributeExpressions(ff).getValue(); + if (md5 != null) { + blobInfoBuilder.setMd5(md5); + blobWriteOptions.add(Storage.BlobWriteOption.md5Match()); + } + + final String crc32c = context.getProperty(CRC32C) + .evaluateAttributeExpressions(ff).getValue(); + if (crc32c != null) { + blobInfoBuilder.setCrc32c(crc32c); + blobWriteOptions.add(Storage.BlobWriteOption.crc32cMatch()); + } + + final String acl = context.getProperty(ACL).getValue(); + if (acl != null) { + blobWriteOptions.add(Storage.BlobWriteOption.predefinedAcl( + Storage.PredefinedAcl.valueOf(acl) + )); + } + + final String encryptionKey = context.getProperty(ENCRYPTION_KEY) + .evaluateAttributeExpressions(ff).getValue(); + if (encryptionKey != null) { + blobWriteOptions.add(Storage.BlobWriteOption.encryptionKey(encryptionKey)); + } + + final HashMap userMetadata = new HashMap<>(); + for (final Map.Entry entry : context.getProperties().entrySet()) { + if (entry.getKey().isDynamic()) { + final String value = context.getProperty( + entry.getKey()).evaluateAttributeExpressions(ff).getValue(); + userMetadata.put(entry.getKey().getName(), value); + } + } + + if (!userMetadata.isEmpty()) { + blobInfoBuilder.setMetadata(userMetadata); + } + + try { + + final Blob blob = storage.create(blobInfoBuilder.build(), + in, + blobWriteOptions.toArray(new Storage.BlobWriteOption[blobWriteOptions.size()]) + ); + + // Create attributes + attributes.put(BUCKET_ATTR, blob.getBucket()); + attributes.put(KEY_ATTR, blob.getName()); + + + if (blob.getSize() != null) { + attributes.put(SIZE_ATTR, String.valueOf(blob.getSize())); + } + + if (blob.getCacheControl() != null) { + attributes.put(CACHE_CONTROL_ATTR, blob.getCacheControl()); + } + + if (blob.getComponentCount() != null) { + attributes.put(COMPONENT_COUNT_ATTR, String.valueOf(blob.getComponentCount())); + } + + if (blob.getContentDisposition() != null) { + attributes.put(CONTENT_DISPOSITION_ATTR, blob.getContentDisposition()); + final Util.ParsedContentDisposition parsed = Util.parseContentDisposition(blob.getContentDisposition()); + + if (parsed != null) { + attributes.put(CoreAttributes.FILENAME.key(), parsed.getFileName()); + } + } + + if (blob.getContentEncoding() != null) { + attributes.put(CONTENT_ENCODING_ATTR, blob.getContentEncoding()); + } + + if (blob.getContentLanguage() != null) { + attributes.put(CONTENT_LANGUAGE_ATTR, blob.getContentLanguage()); + } + + if (blob.getContentType() != null) { + attributes.put(CoreAttributes.MIME_TYPE.key(), blob.getContentType()); + } + + if (blob.getCrc32c() != null) { + attributes.put(CRC32C_ATTR, blob.getCrc32c()); + } + + if (blob.getCustomerEncryption() != null) { + final BlobInfo.CustomerEncryption encryption = blob.getCustomerEncryption(); + + attributes.put(ENCRYPTION_ALGORITHM_ATTR, encryption.getEncryptionAlgorithm()); + attributes.put(ENCRYPTION_SHA256_ATTR, encryption.getKeySha256()); + } + + if (blob.getEtag() != null) { + attributes.put(ETAG_ATTR, blob.getEtag()); + } + + if (blob.getGeneratedId() != null) { + attributes.put(GENERATED_ID_ATTR, blob.getGeneratedId()); + } + + if (blob.getGeneration() != null) { + attributes.put(GENERATION_ATTR, String.valueOf(blob.getGeneration())); + } + + if (blob.getMd5() != null) { + attributes.put(MD5_ATTR, blob.getMd5()); + } + + if (blob.getMediaLink() != null) { + attributes.put(MEDIA_LINK_ATTR, blob.getMediaLink()); + } + + if (blob.getMetageneration() != null) { + attributes.put(METAGENERATION_ATTR, String.valueOf(blob.getMetageneration())); + } + + if (blob.getOwner() != null) { + final Acl.Entity entity = blob.getOwner(); + + if (entity instanceof Acl.User) { + attributes.put(OWNER_ATTR, ((Acl.User) entity).getEmail()); + attributes.put(OWNER_TYPE_ATTR, "user"); + } else if (entity instanceof Acl.Group) { + attributes.put(OWNER_ATTR, ((Acl.Group) entity).getEmail()); + attributes.put(OWNER_TYPE_ATTR, "group"); + } else if (entity instanceof Acl.Domain) { + attributes.put(OWNER_ATTR, ((Acl.Domain) entity).getDomain()); + attributes.put(OWNER_TYPE_ATTR, "domain"); + } else if (entity instanceof Acl.Project) { + attributes.put(OWNER_ATTR, ((Acl.Project) entity).getProjectId()); + attributes.put(OWNER_TYPE_ATTR, "project"); + } + } + + if (blob.getSelfLink() != null) { + attributes.put(URI_ATTR, blob.getSelfLink()); + } + + if (blob.getCreateTime() != null) { + attributes.put(CREATE_TIME_ATTR, String.valueOf(blob.getCreateTime())); + } + + if (blob.getUpdateTime() != null) { + attributes.put(UPDATE_TIME_ATTR, String.valueOf(blob.getUpdateTime())); + } + } catch (StorageException e) { + getLogger().error("Failure completing upload flowfile={} bucket={} key={} reason={}", + new Object[]{ffFilename, bucket, key, e.getMessage()}, e); + throw (e); + } + + + } + } + }); + + if (!attributes.isEmpty()) { + flowFile = session.putAllAttributes(flowFile, attributes); + } + session.transfer(flowFile, REL_SUCCESS); + final long millis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos); + final String url = "https://" + bucket + ".storage.googleapis.com/" + key; + + session.getProvenanceReporter().send(flowFile, url, millis); + getLogger().info("Successfully put {} to Google Cloud Storage in {} milliseconds", + new Object[]{ff, millis}); + + } catch (final ProcessException | StorageException e) { + getLogger().error("Failed to put {} to Google Cloud Storage due to {}", new Object[]{flowFile, e.getMessage()}, e); + flowFile = session.penalize(flowFile); + session.transfer(flowFile, REL_FAILURE); + } + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/StorageAttributes.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/StorageAttributes.java new file mode 100644 index 000000000000..46ef2071fd3b --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/StorageAttributes.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.processors.gcp.storage; + + +/** + * Common attributes being written and accessed through Google Cloud Storage. + */ +public class StorageAttributes { + private StorageAttributes() {} + + public static final String BUCKET_ATTR = "gcs.bucket"; + public static final String BUCKET_DESC = "Bucket of the object."; + + public static final String KEY_ATTR = "gcs.key"; + public static final String KEY_DESC = "Name of the object."; + + public static final String SIZE_ATTR = "gcs.size"; + public static final String SIZE_DESC = "Size of the object."; + + public static final String CACHE_CONTROL_ATTR = "gcs.cache.control"; + public static final String CACHE_CONTROL_DESC = "Data cache control of the object."; + + public static final String COMPONENT_COUNT_ATTR = "gcs.component.count"; + public static final String COMPONENT_COUNT_DESC = "The number of components which make up the object."; + + public static final String CONTENT_DISPOSITION_ATTR = "gcs.content.disposition"; + public static final String CONTENT_DISPOSITION_DESC = "The data content disposition of the object."; + + public static final String CONTENT_ENCODING_ATTR = "gcs.content.encoding"; + public static final String CONTENT_ENCODING_DESC = "The content encoding of the object."; + + public static final String CONTENT_LANGUAGE_ATTR = "gcs.content.language"; + public static final String CONTENT_LANGUAGE_DESC = "The content language of the object."; + + public static final String CRC32C_ATTR = "gcs.crc32c"; + public static final String CRC32C_DESC = "The CRC32C checksum of object's data, encoded in base64 in " + + "big-endian order."; + + public static final String CREATE_TIME_ATTR = "gcs.create.time"; + public static final String CREATE_TIME_DESC = "The creation time of the object (milliseconds)"; + + public static final String DELETE_TIME_ATTR = "gcs.delete.time"; + public static final String DELETE_TIME_DESC = "The deletion time of the object (milliseconds)"; + + public static final String UPDATE_TIME_ATTR = "gcs.update.time"; + public static final String UPDATE_TIME_DESC = "The last modification time of the object (milliseconds)"; + + public static final String ENCRYPTION_ALGORITHM_ATTR = "gcs.encryption.algorithm"; + public static final String ENCRYPTION_ALGORITHM_DESC = "The algorithm used to encrypt the object."; + + public static final String ENCRYPTION_SHA256_ATTR = "gcs.encryption.sha256"; + public static final String ENCRYPTION_SHA256_DESC = "The SHA256 hash of the key used to encrypt the object"; + + public static final String ETAG_ATTR = "gcs.etag"; + public static final String ETAG_DESC = "The HTTP 1.1 Entity tag for the object."; + + public static final String GENERATED_ID_ATTR = "gcs.generated.id"; + public static final String GENERATED_ID_DESC = "The service-generated for the object"; + + public static final String GENERATION_ATTR = "gcs.generation"; + public static final String GENERATION_DESC = "The data generation of the object."; + + public static final String MD5_ATTR = "gcs.md5"; + public static final String MD5_DESC = "The MD5 hash of the object's data encoded in base64."; + + public static final String MEDIA_LINK_ATTR = "gcs.media.link"; + public static final String MEDIA_LINK_DESC = "The media download link to the object."; + + public static final String METAGENERATION_ATTR = "gcs.metageneration"; + public static final String METAGENERATION_DESC = "The metageneration of the object."; + + public static final String OWNER_ATTR = "gcs.owner"; + public static final String OWNER_DESC = "The owner (uploader) of the object."; + + public static final String OWNER_TYPE_ATTR = "gcs.owner.type"; + public static final String OWNER_TYPE_DESC = "The ACL entity type of the uploader of the object."; + + public static final String URI_ATTR = "gcs.uri"; + public static final String URI_DESC = "The URI of the object as a string."; +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/Util.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/Util.java new file mode 100644 index 000000000000..b84a56097cb1 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/storage/Util.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.processors.gcp.storage; + +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Utility class(es) for Storage functionality. + */ +class Util { + + private static final Pattern CONTENT_DISPOSITION_PATTERN = + Pattern.compile("^(.+);\\s*filename\\s*=\\s*\"([^\"]*)\""); + /** + * Parses the filename from a Content-Disposition + * header. + * @param contentDisposition The Content-Disposition header to be parsed + * @return the parsed content disposition. + */ + public static ParsedContentDisposition parseContentDisposition(String contentDisposition) { + Matcher m = CONTENT_DISPOSITION_PATTERN.matcher(contentDisposition); + if (m.find() && m.groupCount() == 2) { + return new ParsedContentDisposition(m.group(1), m.group(2)); + } + return null; + } + + public static class ParsedContentDisposition { + private final String contentDispositionType; + private final String fileName; + + private ParsedContentDisposition(String contentDispositionType, String fileName) { + this.contentDispositionType = contentDispositionType; + this.fileName = fileName; + } + + + public String getFileName() { + return this.fileName; + } + + public String getContentDispositionType() { + return this.contentDispositionType; + } + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService new file mode 100644 index 000000000000..ee70d2fbeffd --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -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.gcp.credentials.service.GCPCredentialsControllerService \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor new file mode 100644 index 000000000000..b5d5df79d0b6 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor @@ -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. +org.apache.nifi.processors.gcp.storage.PutGCSObject +org.apache.nifi.processors.gcp.storage.FetchGCSObject +org.apache.nifi.processors.gcp.storage.DeleteGCSObject +org.apache.nifi.processors.gcp.storage.ListGCSBucket \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/GCPIntegrationTests.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/GCPIntegrationTests.java new file mode 100644 index 000000000000..d259e81dd7ad --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/GCPIntegrationTests.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.gcp; + +/** + * Dummy interface used for annotating Integration Tests. + */ +public interface GCPIntegrationTests { +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.java new file mode 100644 index 000000000000..290ec839abaa --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/CredentialsFactoryTest.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.processors.gcp.credentials.factory; + +import com.google.auth.oauth2.ComputeEngineCredentials; +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ServiceAccountCredentials; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Modifier; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +/** + * Tests of the validation and credentials provider capabilities of CredentialsFactory. + */ +public class CredentialsFactoryTest { + + @Test + public void testCredentialPropertyDescriptorClassCannotBeInvoked() throws Exception { + Constructor constructor = CredentialPropertyDescriptors.class.getDeclaredConstructor(); + assertTrue("Constructor of CredentialPropertyDescriptors should be private", Modifier.isPrivate(constructor.getModifiers())); + constructor.setAccessible(true); + constructor.newInstance(); + } + + @Test + public void testImplicitApplicationDefaultCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.assertValid(); + + Map properties = runner.getProcessContext().getProperties(); + final CredentialsFactory factory = new CredentialsFactory(); + final GoogleCredentials credentials = factory.getGoogleCredentials(properties); + + assertNotNull(credentials); + } + + @Test + public void testExplicitApplicationDefaultCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.setProperty(CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS, "true"); + runner.assertValid(); + + Map properties = runner.getProcessContext().getProperties(); + final CredentialsFactory factory = new CredentialsFactory(); + final GoogleCredentials credentials = factory.getGoogleCredentials(properties); + + assertNotNull(credentials); + } + + @Test + public void testExplicitApplicationDefaultCredentialsExclusive() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.setProperty(CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS, "true"); + runner.setProperty(CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS, "true"); + runner.assertNotValid(); + } + + @Test + public void testJsonFileCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.setProperty(CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE, + "src/test/resources/mock-gcp-service-account.json"); + runner.assertValid(); + + Map properties = runner.getProcessContext().getProperties(); + final CredentialsFactory factory = new CredentialsFactory(); + final GoogleCredentials credentials = factory.getGoogleCredentials(properties); + + assertNotNull(credentials); + assertEquals("credentials class should be equal", ServiceAccountCredentials.class, + credentials.getClass()); + } + + + @Test + public void testBadJsonFileCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.setProperty(CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE, + "src/test/resources/bad-mock-gcp-service-account.json"); + runner.assertNotValid(); + } + + @Test + public void testJsonStringCredentials() throws Exception { + final String jsonRead = new String( + Files.readAllBytes(Paths.get("src/test/resources/mock-gcp-service-account.json")) + ); + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.setProperty(CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON, + jsonRead); + runner.assertValid(); + + Map properties = runner.getProcessContext().getProperties(); + final CredentialsFactory factory = new CredentialsFactory(); + final GoogleCredentials credentials = factory.getGoogleCredentials(properties); + + assertNotNull(credentials); + assertEquals("credentials class should be equal", ServiceAccountCredentials.class, + credentials.getClass()); + } + + @Test + public void testComputeEngineCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsFactoryProcessor.class); + runner.setProperty(CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS, "true"); + runner.assertValid(); + + Map properties = runner.getProcessContext().getProperties(); + final CredentialsFactory factory = new CredentialsFactory(); + final GoogleCredentials credentials = factory.getGoogleCredentials(properties); + + assertNotNull(credentials); + assertEquals("credentials class should be equal", ComputeEngineCredentials.class, + credentials.getClass()); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/MockCredentialsFactoryProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/MockCredentialsFactoryProcessor.java new file mode 100644 index 000000000000..0734c400e323 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/factory/MockCredentialsFactoryProcessor.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.processors.gcp.credentials.factory; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.HttpServiceOptions; +import com.google.cloud.Service; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processors.gcp.AbstractGCPProcessor; + +import java.util.Arrays; +import java.util.Collection; +import java.util.List; + +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +public class MockCredentialsFactoryProcessor extends AbstractGCPProcessor { + public final List properties = Arrays.asList( + USE_APPLICATION_DEFAULT_CREDENTIALS, + SERVICE_ACCOUNT_JSON, + SERVICE_ACCOUNT_JSON_FILE, + USE_COMPUTE_ENGINE_CREDENTIALS + ); + + @Override + public List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + protected HttpServiceOptions getServiceOptions(ProcessContext context, GoogleCredentials credentials) { + HttpServiceOptions mockOptions = mock(HttpServiceOptions.class); + Service mockService = mock(Service.class); + when(mockOptions.getService()).thenReturn(mockService); + + return mockOptions; + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + } + + @Override + protected Collection customValidate(ValidationContext validationContext) { + CredentialsFactory factory = new CredentialsFactory(); + return factory.validate(validationContext); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsServiceTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsServiceTest.java new file mode 100644 index 000000000000..edf0e0a30320 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/service/GCPCredentialsServiceTest.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.processors.gcp.credentials.service; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.auth.oauth2.ServiceAccountCredentials; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Test; + +import java.nio.file.Files; +import java.nio.file.Paths; + +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_APPLICATION_DEFAULT_CREDENTIALS; +import static org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors.USE_COMPUTE_ENGINE_CREDENTIALS; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; + + +public class GCPCredentialsServiceTest { + @Test + public void testToString() throws Exception { + // toString method shouldn't cause an exception + final GCPCredentialsControllerService service = new GCPCredentialsControllerService(); + service.toString(); + } + + @Test + public void testDefaultingToApplicationDefault() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsServiceProcessor.class); + final GCPCredentialsControllerService serviceImpl = new GCPCredentialsControllerService(); + runner.addControllerService("gcpCredentialsProvider", serviceImpl); + + runner.enableControllerService(serviceImpl); + + runner.assertValid(serviceImpl); + + final GCPCredentialsService service = (GCPCredentialsService) runner.getProcessContext() + .getControllerServiceLookup().getControllerService("gcpCredentialsProvider"); + + assertNotNull(service); + final GoogleCredentials credentials = service.getGoogleCredentials(); + assertNotNull(credentials); + } + + @Test + public void testExplicitApplicationDefault() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsServiceProcessor.class); + final GCPCredentialsControllerService serviceImpl = new GCPCredentialsControllerService(); + runner.addControllerService("gcpCredentialsProvider", serviceImpl); + + runner.setProperty(serviceImpl, USE_APPLICATION_DEFAULT_CREDENTIALS, "true"); + runner.enableControllerService(serviceImpl); + + runner.assertValid(serviceImpl); + + final GCPCredentialsService service = (GCPCredentialsService) runner.getProcessContext() + .getControllerServiceLookup().getControllerService("gcpCredentialsProvider"); + + assertNotNull(service); + final GoogleCredentials credentials = service.getGoogleCredentials(); + assertNotNull(credentials); + } + + @Test + public void testFileCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsServiceProcessor.class); + final GCPCredentialsControllerService serviceImpl = new GCPCredentialsControllerService(); + runner.addControllerService("gcpCredentialsProvider", serviceImpl); + + runner.setProperty(serviceImpl, SERVICE_ACCOUNT_JSON_FILE, + "src/test/resources/mock-gcp-service-account.json"); + runner.enableControllerService(serviceImpl); + + runner.assertValid(serviceImpl); + + final GCPCredentialsService service = (GCPCredentialsService) runner.getProcessContext() + .getControllerServiceLookup().getControllerService("gcpCredentialsProvider"); + + assertNotNull(service); + final GoogleCredentials credentials = service.getGoogleCredentials(); + assertNotNull(credentials); + + assertEquals("Credentials class should be equal", ServiceAccountCredentials.class, + credentials.getClass()); + } + + @Test + public void testBadFileCredentials() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsServiceProcessor.class); + final GCPCredentialsControllerService serviceImpl = new GCPCredentialsControllerService(); + runner.addControllerService("gcpCredentialsProvider", serviceImpl); + runner.setProperty(serviceImpl, SERVICE_ACCOUNT_JSON_FILE, + "src/test/resources/bad-mock-gcp-service-account.json"); + runner.assertNotValid(serviceImpl); + } + + @Test + public void testMultipleCredentialSources() throws Exception { + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsServiceProcessor.class); + final GCPCredentialsControllerService serviceImpl = new GCPCredentialsControllerService(); + + runner.addControllerService("gcpCredentialsProvider", serviceImpl); + runner.setProperty(serviceImpl, SERVICE_ACCOUNT_JSON_FILE, + "src/test/resources/mock-gcp-service-account.json"); + runner.setProperty(serviceImpl, USE_APPLICATION_DEFAULT_CREDENTIALS, "true"); + runner.setProperty(serviceImpl, USE_COMPUTE_ENGINE_CREDENTIALS, "true"); + + runner.assertNotValid(serviceImpl); + } + + @Test + public void testRawJsonCredentials() throws Exception { + final String jsonRead = new String( + Files.readAllBytes(Paths.get("src/test/resources/mock-gcp-service-account.json")) + ); + + final TestRunner runner = TestRunners.newTestRunner(MockCredentialsServiceProcessor.class); + final GCPCredentialsControllerService serviceImpl = new GCPCredentialsControllerService(); + runner.addControllerService("gcpCredentialsProvider", serviceImpl); + + runner.setProperty(serviceImpl, SERVICE_ACCOUNT_JSON, + jsonRead); + runner.enableControllerService(serviceImpl); + + runner.assertValid(serviceImpl); + + final GCPCredentialsService service = (GCPCredentialsService) runner.getProcessContext() + .getControllerServiceLookup().getControllerService("gcpCredentialsProvider"); + + assertNotNull(service); + final GoogleCredentials credentials = service.getGoogleCredentials(); + assertNotNull(credentials); + + assertEquals("Credentials class should be equal", ServiceAccountCredentials.class, + credentials.getClass()); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/service/MockCredentialsServiceProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/service/MockCredentialsServiceProcessor.java new file mode 100644 index 000000000000..36c71d610446 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/credentials/service/MockCredentialsServiceProcessor.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.processors.gcp.credentials.service; + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.HttpServiceOptions; +import com.google.cloud.Service; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processors.gcp.AbstractGCPProcessor; + +import java.util.Arrays; +import java.util.List; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + + +public class MockCredentialsServiceProcessor extends AbstractGCPProcessor { + public final List properties = Arrays.asList( + GCP_CREDENTIALS_PROVIDER_SERVICE + ); + + @Override + public List getSupportedPropertyDescriptors() { + return properties; + } + + @Override + protected HttpServiceOptions getServiceOptions(ProcessContext context, GoogleCredentials credentials) { + HttpServiceOptions mockOptions = mock(HttpServiceOptions.class); + Service mockService = mock(Service.class); + when(mockOptions.getService()).thenReturn(mockService); + + return mockOptions; + } + + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/AbstractGCSIT.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/AbstractGCSIT.java new file mode 100644 index 000000000000..725f824d0bd1 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/AbstractGCSIT.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.processors.gcp.storage; + +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.BucketInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.cloud.storage.testing.RemoteStorageHelper; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processors.gcp.GCPIntegrationTests; +import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsControllerService; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +import java.util.Arrays; +import java.util.Iterator; + +import static org.junit.Assert.fail; + +/** + * Base class for GCS Integration Tests. Establishes a bucket and helper methods for creating test scenarios. + * Assumes use of Application Default + * credentials for running tests. + */ +@Category(GCPIntegrationTests.class) +public abstract class AbstractGCSIT { + private static final String PROJECT_ID = System.getProperty("test.gcp.project.id", "nifi-test-gcp-project"); + protected static final String BUCKET = RemoteStorageHelper.generateBucketName(); + protected static final String ENCRYPTION_KEY = "3gCN8OOPAGpDwRYieHAj6fR0eBSG5vloaHl9vlZ3doQ="; + protected static final Integer RETRIES = 6; + + protected static RemoteStorageHelper helper; + protected static Storage storage; + + @BeforeClass + public static void setUp() { + try { + helper = RemoteStorageHelper.create(); + storage = helper.getOptions().getService(); + + if (storage.get(BUCKET) != null) { + // As the generateBucketName function uses a UUID, this should pretty much never happen + fail("Bucket " + BUCKET + " exists. Please rerun the test to generate a new bucket name."); + } + + // Create the bucket + storage.create(BucketInfo.of(BUCKET)); + } catch (StorageException e) { + fail("Can't create bucket " + BUCKET + ": " + e.getLocalizedMessage()); + } + + if (storage.get(BUCKET) == null) { + fail("Setup incomplete, tests will fail"); + } + } + + @AfterClass + public static void tearDown() { + try { + // Empty the bucket before deleting it. + Iterator blobIterator = storage.list(BUCKET, Storage.BlobListOption.versions(true)).iterateAll(); + + while(blobIterator.hasNext()) { + Blob blob = blobIterator.next(); + storage.delete(blob.getBlobId()); + } + + storage.delete(BUCKET); + } catch (final StorageException e) { + fail("Unable to delete bucket " + BUCKET + ": " + e.getLocalizedMessage()); + } + + if (storage.get(BUCKET) != null) { + fail("Incomplete teardown, subsequent tests might fail"); + } + } + + protected static TestRunner buildNewRunner(Processor processor) throws Exception { + final GCPCredentialsControllerService credentialsControllerService = new GCPCredentialsControllerService(); + final TestRunner runner = TestRunners.newTestRunner(processor); + runner.addControllerService("gcpCredentialsControllerService", credentialsControllerService); + runner.enableControllerService(credentialsControllerService); + + runner.setProperty(AbstractGCSProcessor.GCP_CREDENTIALS_PROVIDER_SERVICE, "gcpCredentialsControllerService"); + runner.setProperty(AbstractGCSProcessor.PROJECT_ID, PROJECT_ID); + runner.setProperty(AbstractGCSProcessor.RETRY_COUNT, String.valueOf(RETRIES)); + + runner.assertValid(credentialsControllerService); + + return runner; + } + + /** + * Puts a test file onto Google Cloud Storage in bucket {@link AbstractGCSIT#BUCKET}. + * + * @param key Key which the file will be uploaded under + * @param bytes The content of the file to be uploaded + * @throws StorageException if the file can't be created for some reason + */ + protected void putTestFile(String key, byte[] bytes) throws StorageException { + storage.create(BlobInfo.newBuilder(BlobId.of(BUCKET, key)) + .build(), bytes + ); + } + + /** + * Puts a test file onto Google Cloud Storage in bucket {@link AbstractGCSIT#BUCKET}. This file is encrypted with + * server-side encryption using {@link AbstractGCSIT#ENCRYPTION_KEY}. + * + * @param key Key which the file will be uploaded under + * @param bytes The content of the file to be uploaded + * @throws StorageException if the file can't be created for some reason + */ + protected void putTestFileEncrypted(String key, byte[] bytes) throws StorageException { + storage.create(BlobInfo.newBuilder(BlobId.of(BUCKET, key)) + .build(), bytes, Storage.BlobTargetOption.encryptionKey(ENCRYPTION_KEY)); + } + + /** + * Test if the file exists in Google Cloud Storage in bucket {@link AbstractGCSIT#BUCKET}. + * + * @param key Key to check for the file + * @return true if the file exists, false if it doesn't + * @throws StorageException if there are any issues accessing the file or connecting to GCS. + */ + protected boolean fileExists(String key) throws StorageException { + return (storage.get(BlobId.of(BUCKET, key)) != null); + } + + /** + * Test if the file exists in Google Cloud Storage in bucket {@link AbstractGCSIT#BUCKET}, and if the content is as + * specified. + * + * @param key Key to check for the file + * @param bytes The content to compare to the content of the file + * @return true if the file exists and the content of the file is equal to {@code bytes}, false otherwise. + * @throws StorageException if there are any issues accessing the file or connecting to GCS. + */ + protected boolean fileEquals(String key, byte[] bytes) throws StorageException { + return (fileExists(key) && Arrays.equals(storage.readAllBytes(BlobId.of(BUCKET, key)), bytes)); + } + + /** + * Test if the file exists in Google Cloud Storage in bucket {@link AbstractGCSIT#BUCKET}, and if the content is as + * specified. Assumes that the file is encrypted using {@link AbstractGCSIT#ENCRYPTION_KEY}. + * + * @param key Key to check for the file + * @param bytes The content to compare to the content of the file + * @return true if the file exists and the content of the file is equal to {@code bytes}, false otherwise. + * @throws StorageException if there are any issues accessing the file or connecting to GCS. + */ + protected boolean fileEqualsEncrypted(String key, byte[] bytes) throws StorageException { + return (fileExists(key) && Arrays.equals( + storage.readAllBytes(BlobId.of(BUCKET, key), Storage.BlobSourceOption.decryptionKey(ENCRYPTION_KEY)), + bytes)); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/AbstractGCSTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/AbstractGCSTest.java new file mode 100644 index 000000000000..730cd85e1a42 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/AbstractGCSTest.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.processors.gcp.storage; + + +import com.google.auth.oauth2.GoogleCredentials; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; +import com.google.cloud.storage.testing.RemoteStorageHelper; +import org.apache.nifi.processor.Processor; +import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsControllerService; +import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsService; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertSame; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; + +/** + * Base class for GCS Unit Tests. Provides a framework for creating a TestRunner instance with always-required credentials. + */ +public abstract class AbstractGCSTest { + private static final String PROJECT_ID = System.getProperty("test.gcp.project.id", "nifi-test-gcp-project"); + private static final Integer RETRIES = 9; + + static final String BUCKET = RemoteStorageHelper.generateBucketName(); + + @Before + public void setup() throws Exception { + MockitoAnnotations.initMocks(this); + } + + public static TestRunner buildNewRunner(Processor processor) throws Exception { + final GCPCredentialsService credentialsService = new GCPCredentialsControllerService(); + + final TestRunner runner = TestRunners.newTestRunner(processor); + runner.addControllerService("gcpCredentialsControllerService", credentialsService); + runner.enableControllerService(credentialsService); + + runner.setProperty(AbstractGCSProcessor.GCP_CREDENTIALS_PROVIDER_SERVICE, "gcpCredentialsControllerService"); + runner.setProperty(AbstractGCSProcessor.PROJECT_ID, PROJECT_ID); + runner.setProperty(AbstractGCSProcessor.RETRY_COUNT, String.valueOf(RETRIES)); + + runner.assertValid(credentialsService); + + return runner; + } + + public abstract AbstractGCSProcessor getProcessor(); + + protected abstract void addRequiredPropertiesToRunner(TestRunner runner); + + @Mock + protected Storage storage; + + @Test + public void testStorageOptionsConfiguration() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + + final AbstractGCSProcessor processor = getProcessor(); + final GoogleCredentials mockCredentials = mock(GoogleCredentials.class); + + final StorageOptions options = processor.getServiceOptions(runner.getProcessContext(), + mockCredentials); + + assertEquals("Project IDs should match", + PROJECT_ID, options.getProjectId()); + + assertEquals("Retry counts should match", + RETRIES.intValue(), options.getRetryParams().getRetryMinAttempts()); + + assertEquals("Retry counts should match", + RETRIES.intValue(), options.getRetryParams().getRetryMaxAttempts()); + + assertSame("Credentials should be configured correctly", + mockCredentials, options.getCredentials()); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObjectIT.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObjectIT.java new file mode 100644 index 000000000000..8beb521ce25a --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObjectIT.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.processors.gcp.storage; + +import com.google.common.collect.ImmutableMap; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.util.TestRunner; +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +/** + * Integration tests for {@link DeleteGCSObject} which actually use Google Cloud resources. + */ +public class DeleteGCSObjectIT extends AbstractGCSIT { + static final String KEY = "delete-me"; + + @Test + public void testSimpleDeleteWithFilename() throws Exception { + putTestFile(KEY, new byte[]{7, 8, 9}); + assertTrue(fileExists(KEY)); + + final TestRunner runner = buildNewRunner(new DeleteGCSObject()); + runner.setProperty(DeleteGCSObject.BUCKET, BUCKET); + runner.assertValid(); + + runner.enqueue("testdata", ImmutableMap.of( + CoreAttributes.FILENAME.key(), KEY + )); + + runner.run(); + + runner.assertAllFlowFilesTransferred(DeleteGCSObject.REL_SUCCESS); + runner.assertTransferCount(DeleteGCSObject.REL_SUCCESS, 1); + + assertFalse(fileExists(KEY)); + } + + @Test + public void testSimpleDeleteWithPropertySet() throws Exception { + putTestFile(KEY, new byte[]{7, 8, 9}); + assertTrue(fileExists(KEY)); + + final TestRunner runner = buildNewRunner(new DeleteGCSObject()); + runner.setProperty(DeleteGCSObject.BUCKET, BUCKET); + runner.setProperty(DeleteGCSObject.KEY, KEY); + runner.assertValid(); + + runner.enqueue("testdata", ImmutableMap.of( + "filename", "different-filename" + )); + + runner.run(); + + runner.assertAllFlowFilesTransferred(DeleteGCSObject.REL_SUCCESS); + runner.assertTransferCount(DeleteGCSObject.REL_SUCCESS, 1); + + assertFalse(fileExists(KEY)); + } + + @Test + public void testDeleteNonExistentFile() throws Exception { + final TestRunner runner = buildNewRunner(new DeleteGCSObject()); + runner.setProperty(DeleteGCSObject.BUCKET, BUCKET); + runner.assertValid(); + + runner.enqueue("testdata", ImmutableMap.of( + "filename", "nonexistant-file" + )); + + runner.run(); + + runner.assertAllFlowFilesTransferred(DeleteGCSObject.REL_SUCCESS); + runner.assertTransferCount(DeleteGCSObject.REL_SUCCESS, 1); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObjectTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObjectTest.java new file mode 100644 index 000000000000..bbf97b80a41f --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/DeleteGCSObjectTest.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.processors.gcp.storage; + +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.common.collect.ImmutableMap; +import org.apache.nifi.util.TestRunner; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link DeleteGCSObject}. No connections to the Google Cloud service are made. + */ +public class DeleteGCSObjectTest extends AbstractGCSTest { + public static final Long GENERATION = 42L; + static final String KEY = "somefile"; + + + public static final String BUCKET_ATTR = "gcs.bucket"; + public static final String KEY_ATTR = "gcs.key"; + public static final String GENERATION_ATTR = "gcs.generation"; + + @Mock + Storage storage; + + @Before + public void setup() throws Exception { + MockitoAnnotations.initMocks(this); + } + + @Override + protected void addRequiredPropertiesToRunner(TestRunner runner) { + runner.setProperty(DeleteGCSObject.BUCKET, BUCKET); + runner.setProperty(DeleteGCSObject.GENERATION, String.valueOf(GENERATION)); + runner.setProperty(DeleteGCSObject.KEY, KEY); + } + + @Override + public DeleteGCSObject getProcessor() { + return new DeleteGCSObject() { + @Override + protected Storage getCloudService() { + return storage; + } + }; + } + + + @Test + public void testDeleteWithValidArguments() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + + runner.assertValid(); + + runner.enqueue("testdata"); + + runner.run(); + + verify(storage).delete(eq(BlobId.of(BUCKET, KEY, GENERATION))); + + runner.assertAllFlowFilesTransferred(DeleteGCSObject.REL_SUCCESS); + runner.assertTransferCount(DeleteGCSObject.REL_SUCCESS, 1); + } + + @Test + public void testTwoDeletesWithFlowfileAttributes() throws Exception { + reset(storage); + + final TestRunner runner = buildNewRunner(getProcessor()); + + runner.setProperty(DeleteGCSObject.BUCKET, "${" + BUCKET_ATTR + "}"); + runner.setProperty(DeleteGCSObject.KEY, "${" + KEY_ATTR + "}"); + runner.setProperty(DeleteGCSObject.GENERATION, "${" + GENERATION_ATTR + "}"); + + runner.assertValid(); + + final String bucket1 = BUCKET + "_1"; + final String bucket2 = BUCKET + "_2"; + final String key1 = KEY + "_1"; + final String key2 = KEY + "_2"; + final Long generation1 = GENERATION + 1L; + final Long generation2 = GENERATION + 2L; + + runner.enqueue("testdata1", ImmutableMap.of( + BUCKET_ATTR, bucket1, + KEY_ATTR, key1, + GENERATION_ATTR, String.valueOf(generation1) + )); + + runner.enqueue("testdata2", ImmutableMap.of( + BUCKET_ATTR, bucket2, + KEY_ATTR, key2, + GENERATION_ATTR, String.valueOf(generation2) + )); + + runner.run(2); + + verify(storage).delete(eq(BlobId.of(bucket1, key1, generation1))); + verify(storage).delete(eq(BlobId.of(bucket2, key2, generation2))); + + runner.assertAllFlowFilesTransferred(DeleteGCSObject.REL_SUCCESS); + runner.assertTransferCount(DeleteGCSObject.REL_SUCCESS, 2); + } + + @Test + public void testFailureOnException() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + runner.enqueue("testdata"); + + when(storage.delete(any(BlobId.class))).thenThrow(new StorageException(1, "Test Exception")); + + runner.run(); + + runner.assertPenalizeCount(1); + runner.assertAllFlowFilesTransferred(DeleteGCSObject.REL_FAILURE); + runner.assertTransferCount(DeleteGCSObject.REL_FAILURE, 1); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/FetchGCSObjectIT.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/FetchGCSObjectIT.java new file mode 100644 index 000000000000..a8b8306b00c5 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/FetchGCSObjectIT.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.processors.gcp.storage; + +import com.google.common.collect.ImmutableMap; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.junit.Test; + +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertTrue; + +/** + * Integration tests for {@link FetchGCSObject} which actually use Google Cloud resources. + */ +public class FetchGCSObjectIT extends AbstractGCSIT { + static final String KEY = "delete-me"; + static final byte[] CONTENT = {10,11,12}; + + @Test + public void testSimpleFetch() throws Exception { + putTestFile(KEY, CONTENT); + assertTrue(fileExists(KEY)); + + final TestRunner runner = buildNewRunner(new FetchGCSObject()); + runner.setProperty(FetchGCSObject.BUCKET, BUCKET); + + runner.enqueue(new byte[0], ImmutableMap.of( + "filename", KEY + )); + + runner.run(); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS, 1); + final List ffs = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS); + MockFlowFile ff = ffs.get(0); + + ff.assertContentEquals(CONTENT); + + ff.assertAttributeNotExists(StorageAttributes.ENCRYPTION_ALGORITHM_ATTR); + ff.assertAttributeNotExists(StorageAttributes.ENCRYPTION_SHA256_ATTR); + + for (final Map.Entry entry : ff.getAttributes().entrySet()) { + System.out.println(entry.getKey() + ":" + entry.getValue()); + } + } + + @Test + public void testSimpleFetchEncrypted() throws Exception { + putTestFileEncrypted(KEY, CONTENT); + assertTrue(fileExists(KEY)); + + final TestRunner runner = buildNewRunner(new FetchGCSObject()); + runner.setProperty(FetchGCSObject.BUCKET, BUCKET); + runner.setProperty(FetchGCSObject.ENCRYPTION_KEY, ENCRYPTION_KEY); + + runner.enqueue(new byte[0], ImmutableMap.of( + "filename", KEY + )); + + runner.assertValid(); + runner.run(); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS, 1); + final List ffs = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS); + MockFlowFile ff = ffs.get(0); + + ff.assertAttributeEquals(StorageAttributes.ENCRYPTION_ALGORITHM_ATTR, + "AES256"); + } + + + @Test + public void testFetchNonexistantFile() throws Exception { + final TestRunner runner = buildNewRunner(new FetchGCSObject()); + runner.setProperty(FetchGCSObject.BUCKET, BUCKET); + runner.enqueue(new byte[0], ImmutableMap.of( + "filename", "non-existent" + )); + + runner.assertValid(); + runner.run(); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_FAILURE, 1); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/FetchGCSObjectTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/FetchGCSObjectTest.java new file mode 100644 index 000000000000..0d8b20952d53 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/FetchGCSObjectTest.java @@ -0,0 +1,614 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.ReadChannel; +import com.google.cloud.RestorableState; +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.junit.Before; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Set; + +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_ATTR; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link FetchGCSObject}. + */ +public class FetchGCSObjectTest extends AbstractGCSTest { + private final static String KEY = "test-key"; + private final static Long GENERATION = 5L; + private static final String CONTENT = "test-content"; + + private static final Long SIZE = 100L; + private static final String CACHE_CONTROL = "test-cache-control"; + private static final Integer COMPONENT_COUNT = 3; + private static final String CONTENT_ENCODING = "test-content-encoding"; + private static final String CONTENT_LANGUAGE = "test-content-language"; + private static final String CONTENT_TYPE = "test-content-type"; + private static final String CRC32C = "test-crc32c"; + private static final String ENCRYPTION = "test-encryption"; + private static final String ENCRYPTION_SHA256 = "test-encryption-256"; + private static final String ETAG = "test-etag"; + private static final String GENERATED_ID = "test-generated-id"; + private static final String MD5 = "test-md5"; + private static final String MEDIA_LINK = "test-media-link"; + private static final Long METAGENERATION = 42L; + private static final String OWNER_USER_EMAIL = "test-owner-user-email"; + private static final String OWNER_GROUP_EMAIL = "test-owner-group-email"; + private static final String OWNER_DOMAIN = "test-owner-domain"; + private static final String OWNER_PROJECT_ID = "test-owner-project-id"; + private static final String URI = "test-uri"; + private static final String CONTENT_DISPOSITION = "attachment; filename=\"test-content-disposition.txt\""; + private static final Long CREATE_TIME = 1234L; + private static final Long UPDATE_TIME = 4567L; + + + @Mock + Storage storage; + + @Before + public void setup() throws Exception { + MockitoAnnotations.initMocks(this); + } + + @Override + public AbstractGCSProcessor getProcessor() { + return new FetchGCSObject() { + @Override + protected Storage getCloudService() { + return storage; + } + }; + } + + private class MockReadChannel implements ReadChannel { + private byte[] toRead; + private int position = 0; + private boolean finished; + private boolean isOpen; + + private MockReadChannel(String textToRead) { + this.toRead = textToRead.getBytes(); + this.isOpen = true; + this.finished = false; + } + + @Override + public void close() { + this.isOpen = false; + } + + @Override + public void seek(long l) throws IOException { + + } + + @Override + public void chunkSize(int i) { + + } + + @Override + public void setChunkSize(int i) { + + } + + @Override + public RestorableState capture() { + return null; + } + + @Override + public int read(ByteBuffer dst) throws IOException { + if (this.finished) { + return -1; + } else { + if (dst.remaining() > this.toRead.length) { + this.finished = true; + } + int toWrite = Math.min(this.toRead.length - position, dst.remaining()); + + dst.put(this.toRead, this.position, toWrite); + this.position += toWrite; + + return toWrite; + } + } + + @Override + public boolean isOpen() { + return this.isOpen; + } + } + + @Override + protected void addRequiredPropertiesToRunner(TestRunner runner) { + runner.setProperty(FetchGCSObject.BUCKET, BUCKET); + runner.setProperty(FetchGCSObject.KEY, String.valueOf(KEY)); + } + + @Test + public void testSuccessfulFetch() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + + when(blob.getBucket()).thenReturn(BUCKET); + when(blob.getName()).thenReturn(KEY); + when(blob.getSize()).thenReturn(SIZE); + when(blob.getCacheControl()).thenReturn(CACHE_CONTROL); + when(blob.getComponentCount()).thenReturn(COMPONENT_COUNT); + when(blob.getContentEncoding()).thenReturn(CONTENT_ENCODING); + when(blob.getContentLanguage()).thenReturn(CONTENT_LANGUAGE); + when(blob.getContentType()).thenReturn(CONTENT_TYPE); + when(blob.getCrc32c()).thenReturn(CRC32C); + + final BlobInfo.CustomerEncryption mockEncryption = mock(BlobInfo.CustomerEncryption.class); + when(mockEncryption.getEncryptionAlgorithm()).thenReturn(ENCRYPTION); + when(mockEncryption.getKeySha256()).thenReturn(ENCRYPTION_SHA256); + when(blob.getCustomerEncryption()).thenReturn(mockEncryption); + + when(blob.getEtag()).thenReturn(ETAG); + when(blob.getGeneratedId()).thenReturn(GENERATED_ID); + when(blob.getGeneration()).thenReturn(GENERATION); + when(blob.getMd5()).thenReturn(MD5); + when(blob.getMediaLink()).thenReturn(MEDIA_LINK); + when(blob.getMetageneration()).thenReturn(METAGENERATION); + when(blob.getSelfLink()).thenReturn(URI); + when(blob.getContentDisposition()).thenReturn(CONTENT_DISPOSITION); + when(blob.getCreateTime()).thenReturn(CREATE_TIME); + when(blob.getUpdateTime()).thenReturn(UPDATE_TIME); + + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + + runner.enqueue(""); + + runner.run(); + + verify(storage).get(any(BlobId.class)); + verify(storage).reader(any(BlobId.class), any(Storage.BlobSourceOption.class)); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + + assertTrue(flowFile.isContentEqual(CONTENT)); + assertEquals( + BUCKET, + flowFile.getAttribute(BUCKET_ATTR) + ); + + assertEquals( + KEY, + flowFile.getAttribute(KEY_ATTR) + ); + + assertEquals( + CACHE_CONTROL, + flowFile.getAttribute(CACHE_CONTROL_ATTR) + ); + + assertEquals( + COMPONENT_COUNT, + Integer.valueOf(flowFile.getAttribute(COMPONENT_COUNT_ATTR)) + ); + + assertEquals( + CONTENT_ENCODING, + flowFile.getAttribute(CONTENT_ENCODING_ATTR) + ); + + assertEquals( + CONTENT_LANGUAGE, + flowFile.getAttribute(CONTENT_LANGUAGE_ATTR) + ); + + assertEquals( + CONTENT_TYPE, + flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()) + ); + + assertEquals( + CRC32C, + flowFile.getAttribute(CRC32C_ATTR) + ); + + assertEquals( + ENCRYPTION, + flowFile.getAttribute(ENCRYPTION_ALGORITHM_ATTR) + ); + + assertEquals( + ENCRYPTION_SHA256, + flowFile.getAttribute(ENCRYPTION_SHA256_ATTR) + ); + + assertEquals( + ETAG, + flowFile.getAttribute(ETAG_ATTR) + ); + + assertEquals( + GENERATED_ID, + flowFile.getAttribute(GENERATED_ID_ATTR) + ); + + assertEquals( + GENERATION, + Long.valueOf(flowFile.getAttribute(GENERATION_ATTR)) + ); + + assertEquals( + MD5, + flowFile.getAttribute(MD5_ATTR) + ); + + assertEquals( + MEDIA_LINK, + flowFile.getAttribute(MEDIA_LINK_ATTR) + ); + + assertEquals( + METAGENERATION, + Long.valueOf(flowFile.getAttribute(METAGENERATION_ATTR)) + ); + + assertEquals( + URI, + flowFile.getAttribute(URI_ATTR) + ); + + assertEquals( + CONTENT_DISPOSITION, + flowFile.getAttribute(CONTENT_DISPOSITION_ATTR) + ); + + assertEquals( + CREATE_TIME, + Long.valueOf(flowFile.getAttribute(CREATE_TIME_ATTR)) + ); + + assertEquals( + UPDATE_TIME, + Long.valueOf(flowFile.getAttribute(UPDATE_TIME_ATTR)) + ); + + } + + @Test + public void testAclOwnerUser() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + + final Acl.User mockUser = mock(Acl.User.class); + when(mockUser.getEmail()).thenReturn(OWNER_USER_EMAIL); + when(blob.getOwner()).thenReturn(mockUser); + + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + runner.enqueue(""); + + runner.run(); + + verify(storage).get(any(BlobId.class)); + verify(storage).reader(any(BlobId.class), any(Storage.BlobSourceOption.class)); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + + assertEquals( + OWNER_USER_EMAIL, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "user", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + } + + @Test + public void testAclOwnerGroup() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + + final Acl.Group mockGroup = mock(Acl.Group.class); + when(mockGroup.getEmail()).thenReturn(OWNER_GROUP_EMAIL); + when(blob.getOwner()).thenReturn(mockGroup); + + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + + runner.enqueue(""); + + runner.run(); + + verify(storage).get(any(BlobId.class)); + verify(storage).reader(any(BlobId.class), any(Storage.BlobSourceOption.class)); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + + assertEquals( + OWNER_GROUP_EMAIL, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "group", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + } + + + @Test + public void testAclOwnerDomain() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + + final Acl.Domain mockDomain = mock(Acl.Domain.class); + when(mockDomain.getDomain()).thenReturn(OWNER_DOMAIN); + when(blob.getOwner()).thenReturn(mockDomain); + + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + + runner.enqueue(""); + + runner.run(); + + verify(storage).get(any(BlobId.class)); + verify(storage).reader(any(BlobId.class), any(Storage.BlobSourceOption.class)); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + + assertEquals( + OWNER_DOMAIN, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "domain", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + } + + + @Test + public void testAclOwnerProject() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + final Acl.Project mockProject = mock(Acl.Project.class); + when(mockProject.getProjectId()).thenReturn(OWNER_PROJECT_ID); + when(blob.getOwner()).thenReturn(mockProject); + + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + runner.enqueue(""); + + runner.run(); + + verify(storage).get(any(BlobId.class)); + verify(storage).reader(any(BlobId.class), any(Storage.BlobSourceOption.class)); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + + assertEquals( + OWNER_PROJECT_ID, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "project", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + } + + @Test + public void testBlobIdWithGeneration() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + + runner.removeProperty(FetchGCSObject.KEY); + runner.removeProperty(FetchGCSObject.BUCKET); + + runner.setProperty(FetchGCSObject.GENERATION, String.valueOf(GENERATION)); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + runner.enqueue("", ImmutableMap.of( + BUCKET_ATTR, BUCKET, + CoreAttributes.FILENAME.key(), KEY + )); + + runner.run(); + + ArgumentCaptor blobIdArgumentCaptor = ArgumentCaptor.forClass(BlobId.class); + ArgumentCaptor blobSourceOptionArgumentCaptor = ArgumentCaptor.forClass(Storage.BlobSourceOption.class); + verify(storage).get(blobIdArgumentCaptor.capture()); + verify(storage).reader(any(BlobId.class), blobSourceOptionArgumentCaptor.capture()); + + final BlobId blobId = blobIdArgumentCaptor.getValue(); + + assertEquals( + BUCKET, + blobId.getBucket() + ); + + assertEquals( + KEY, + blobId.getName() + ); + + assertEquals( + GENERATION, + blobId.getGeneration() + ); + + + final Set blobSourceOptions = ImmutableSet.copyOf(blobSourceOptionArgumentCaptor.getAllValues()); + assertTrue(blobSourceOptions.contains(Storage.BlobSourceOption.generationMatch())); + assertEquals( + 1, + blobSourceOptions.size() + ); + + } + + + @Test + public void testBlobIdWithEncryption() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + + runner.setProperty(FetchGCSObject.ENCRYPTION_KEY, ENCRYPTION_SHA256); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = mock(Blob.class); + when(storage.get(any(BlobId.class))).thenReturn(blob); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + runner.enqueue(""); + + runner.run(); + + ArgumentCaptor blobIdArgumentCaptor = ArgumentCaptor.forClass(BlobId.class); + ArgumentCaptor blobSourceOptionArgumentCaptor = ArgumentCaptor.forClass(Storage.BlobSourceOption.class); + verify(storage).get(blobIdArgumentCaptor.capture()); + verify(storage).reader(any(BlobId.class), blobSourceOptionArgumentCaptor.capture()); + + final BlobId blobId = blobIdArgumentCaptor.getValue(); + + assertEquals( + BUCKET, + blobId.getBucket() + ); + + assertEquals( + KEY, + blobId.getName() + ); + + assertNull(blobId.getGeneration()); + + final Set blobSourceOptions = ImmutableSet.copyOf(blobSourceOptionArgumentCaptor.getAllValues()); + + assertTrue(blobSourceOptions.contains(Storage.BlobSourceOption.decryptionKey(ENCRYPTION_SHA256))); + assertEquals( + 1, + blobSourceOptions.size() + ); + } + + @Test + public void testStorageExceptionOnFetch() throws Exception { + reset(storage); + final TestRunner runner = buildNewRunner(getProcessor()); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.get(any(BlobId.class))).thenThrow(new StorageException(400, "test-exception")); + when(storage.reader(any(BlobId.class), any(Storage.BlobSourceOption.class))).thenReturn(new MockReadChannel(CONTENT)); + + runner.enqueue(""); + + runner.run(); + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_FAILURE); + runner.assertTransferCount(FetchGCSObject.REL_FAILURE, 1); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/ListGCSBucketIT.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/ListGCSBucketIT.java new file mode 100644 index 000000000000..497e8e0e6a8c --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/ListGCSBucketIT.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.processors.gcp.storage; + +import com.google.cloud.storage.BucketInfo; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.junit.Test; + +import java.util.List; + +import static org.junit.Assert.assertNotEquals; + +/** + * Integration tests for {@link ListGCSBucket} which actually use Google Cloud resources. + */ +public class ListGCSBucketIT extends AbstractGCSIT { + private static final byte[] CONTENT = {12, 13, 14}; + + @Test + public void testSimpleList() throws Exception { + putTestFile("a", CONTENT); + putTestFile("b/c", CONTENT); + putTestFile("d/e", CONTENT); + + final TestRunner runner = buildNewRunner(new ListGCSBucket()); + runner.setProperty(ListGCSBucket.BUCKET, BUCKET); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 3); + List flowFiles = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS); + flowFiles.get(0).assertAttributeEquals("filename", "a"); + flowFiles.get(1).assertAttributeEquals("filename", "b/c"); + flowFiles.get(2).assertAttributeEquals("filename", "d/e"); + } + + + @Test + public void testSimpleListWithPrefix() throws Exception { + putTestFile("a", CONTENT); + putTestFile("b/c", CONTENT); + putTestFile("d/e", CONTENT); + + final TestRunner runner = buildNewRunner(new ListGCSBucket()); + runner.setProperty(ListGCSBucket.BUCKET, BUCKET); + + runner.setProperty(ListGCSBucket.PREFIX, "b/"); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 1); + List flowFiles = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS); + flowFiles.get(0).assertAttributeEquals("filename", "b/c"); + } + + + + @Test + public void testSimpleListWithPrefixAndGenerations() throws Exception { + // enable versioning + storage.update(BucketInfo.newBuilder(BUCKET).setVersioningEnabled(true).build()); + + putTestFile("generations/a", CONTENT); + putTestFile("generations/a", CONTENT); + putTestFile("generations/b", CONTENT); + putTestFile("generations/c", CONTENT); + + final TestRunner runner = buildNewRunner(new ListGCSBucket()); + runner.setProperty(ListGCSBucket.BUCKET, BUCKET); + + runner.setProperty(ListGCSBucket.PREFIX, "generations/"); + runner.setProperty(ListGCSBucket.USE_GENERATIONS, "true"); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 4); + List flowFiles = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS); + flowFiles.get(0).assertAttributeEquals("filename", "generations/a"); + flowFiles.get(1).assertAttributeEquals("filename", "generations/a"); + flowFiles.get(2).assertAttributeEquals("filename", "generations/b"); + flowFiles.get(3).assertAttributeEquals("filename", "generations/c"); + + assertNotEquals( + flowFiles.get(0).getAttribute(StorageAttributes.GENERATION_ATTR), + flowFiles.get(1).getAttribute(StorageAttributes.GENERATION_ATTR) + ); + } + + + @Test + public void testCheckpointing() throws Exception { + putTestFile("checkpoint/a", CONTENT); + putTestFile("checkpoint/b/c", CONTENT); + + final TestRunner runner = buildNewRunner(new ListGCSBucket()); + runner.setProperty(ListGCSBucket.BUCKET, BUCKET); + + runner.setProperty(ListGCSBucket.PREFIX, "checkpoint/"); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 2); + List flowFiles = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS); + flowFiles.get(0).assertAttributeEquals("filename", "checkpoint/a"); + flowFiles.get(1).assertAttributeEquals("filename", "checkpoint/b/c"); + + putTestFile("checkpoint/d/e", CONTENT); + runner.run(); + + // Should only retrieve 1 new file (for a total of 3) + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 3); + flowFiles = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS); + flowFiles.get(0).assertAttributeEquals("filename", "checkpoint/a"); + flowFiles.get(1).assertAttributeEquals("filename", "checkpoint/b/c"); + flowFiles.get(2).assertAttributeEquals("filename", "checkpoint/d/e"); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/ListGCSBucketTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/ListGCSBucketTest.java new file mode 100644 index 000000000000..6c21d913fbae --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/ListGCSBucketTest.java @@ -0,0 +1,827 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.Page; +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.nifi.components.state.Scope; +import org.apache.nifi.components.state.StateMap; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.util.LogMessage; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; + +import java.util.List; +import java.util.Map; + +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_ATTR; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link ListGCSBucket} which do not consume Google Cloud resources. + */ +public class ListGCSBucketTest extends AbstractGCSTest { + private static final String PREFIX = "test-prefix"; + private static final Boolean USE_GENERATIONS = true; + + private static final Long SIZE = 100L; + private static final String CACHE_CONTROL = "test-cache-control"; + private static final Integer COMPONENT_COUNT = 3; + private static final String CONTENT_ENCODING = "test-content-encoding"; + private static final String CONTENT_LANGUAGE = "test-content-language"; + private static final String CONTENT_TYPE = "test-content-type"; + private static final String CRC32C = "test-crc32c"; + private static final String ENCRYPTION = "test-encryption"; + private static final String ENCRYPTION_SHA256 = "test-encryption-256"; + private static final String ETAG = "test-etag"; + private static final String GENERATED_ID = "test-generated-id"; + private static final String MD5 = "test-md5"; + private static final String MEDIA_LINK = "test-media-link"; + private static final Long METAGENERATION = 42L; + private static final String OWNER_USER_EMAIL = "test-owner-user-email"; + private static final String OWNER_GROUP_EMAIL = "test-owner-group-email"; + private static final String OWNER_DOMAIN = "test-owner-domain"; + private static final String OWNER_PROJECT_ID = "test-owner-project-id"; + private static final String URI = "test-uri"; + private static final String CONTENT_DISPOSITION = "attachment; filename=\"test-content-disposition.txt\""; + private static final Long CREATE_TIME = 1234L; + private static final Long UPDATE_TIME = 4567L; + private final static Long GENERATION = 5L; + + @Mock + Storage storage; + + @Captor + ArgumentCaptor argumentCaptor; + + @Override + public ListGCSBucket getProcessor() { + return new ListGCSBucket() { + @Override + protected Storage getCloudService() { + return storage; + } + }; + } + + @Override + protected void addRequiredPropertiesToRunner(TestRunner runner) { + runner.setProperty(ListGCSBucket.BUCKET, BUCKET); + } + + @Test + public void testRestoreFreshState() throws Exception { + reset(storage); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + assertEquals("Cluster StateMap should be fresh (version -1L)", + -1L, + runner.getProcessContext().getStateManager().getState(Scope.CLUSTER).getVersion() + ); + + assertNull(processor.currentKeys); + + processor.restoreState(runner.getProcessContext()); + + assertNotNull(processor.currentKeys); + assertEquals( + 0L, + processor.currentTimestamp + ); + + assertTrue(processor.currentKeys.isEmpty()); + + } + + @Test + public void testRestorePreviousState() throws Exception { + reset(storage); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Map state = ImmutableMap.of( + ListGCSBucket.CURRENT_TIMESTAMP, String.valueOf(4L), + ListGCSBucket.CURRENT_KEY_PREFIX + "0", "test-key-0", + ListGCSBucket.CURRENT_KEY_PREFIX + "1", "test-key-1" + ); + + runner.getStateManager().setState(state, Scope.CLUSTER); + + assertNull(processor.currentKeys); + assertEquals( + 0L, + processor.currentTimestamp + ); + + processor.restoreState(runner.getProcessContext()); + + assertNotNull(processor.currentKeys); + assertTrue(processor.currentKeys.contains("test-key-0")); + assertTrue(processor.currentKeys.contains("test-key-1")); + assertEquals( + 4L, + processor.currentTimestamp + ); + } + + @Test + public void testPersistState() throws Exception { + reset(storage); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + assertEquals("Cluster StateMap should be fresh (version -1L)", + -1L, + runner.getProcessContext().getStateManager().getState(Scope.CLUSTER).getVersion() + ); + + processor.currentKeys = ImmutableSet.of( + "test-key-0", + "test-key-1" + ); + + processor.currentTimestamp = 4L; + + processor.persistState(runner.getProcessContext()); + + final StateMap stateMap = runner.getStateManager().getState(Scope.CLUSTER); + assertEquals( + "Cluster StateMap should have been written to", + 1L, + stateMap.getVersion() + ); + + assertEquals( + ImmutableMap.of( + ListGCSBucket.CURRENT_TIMESTAMP, String.valueOf(4L), + ListGCSBucket.CURRENT_KEY_PREFIX+"0", "test-key-0", + ListGCSBucket.CURRENT_KEY_PREFIX+"1", "test-key-1" + ), + stateMap.toMap() + ); + } + + @Test + public void testFailedPersistState() throws Exception { + reset(storage); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + runner.getStateManager().setFailOnStateSet(Scope.CLUSTER, true); + + processor.currentKeys = ImmutableSet.of( + "test-key-0", + "test-key-1" + ); + + processor.currentTimestamp = 4L; + + assertTrue(runner.getLogger().getErrorMessages().isEmpty()); + + processor.persistState(runner.getProcessContext()); + + // The method should have caught the error and reported it to the logger. + final List logMessages = runner.getLogger().getErrorMessages(); + assertFalse(logMessages.isEmpty()); + assertEquals( + 1, + logMessages.size() + ); + + // We could do more specific things like check the contents of the LogMessage, + // but that seems too nitpicky. + + } + + @Mock + Page mockBlobPages; + + private Blob buildMockBlob(String bucket, String key, long updateTime) { + final Blob blob = mock(Blob.class); + when(blob.getBucket()).thenReturn(bucket); + when(blob.getName()).thenReturn(key); + when(blob.getUpdateTime()).thenReturn(updateTime); + return blob; + } + + @Test + public void testSuccessfulList() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Iterable mockList = ImmutableList.of( + buildMockBlob("blob-bucket-1", "blob-key-1", 2L), + buildMockBlob("blob-bucket-2", "blob-key-2", 3L) + ); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS); + runner.assertTransferCount(ListGCSBucket.REL_SUCCESS, 2); + + final List successes = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS); + + MockFlowFile flowFile = successes.get(0); + assertEquals( + "blob-bucket-1", + flowFile.getAttribute(BUCKET_ATTR) + ); + + assertEquals( + "blob-key-1", + flowFile.getAttribute(KEY_ATTR) + ); + + assertEquals( + "2", + flowFile.getAttribute(UPDATE_TIME_ATTR) + ); + + flowFile = successes.get(1); + assertEquals( + "blob-bucket-2", + flowFile.getAttribute(BUCKET_ATTR) + ); + + assertEquals( + "blob-key-2", + flowFile.getAttribute(KEY_ATTR) + ); + + assertEquals( + "3", + flowFile.getAttribute(UPDATE_TIME_ATTR) + ); + + assertEquals( + 3L, + processor.currentTimestamp + ); + + assertEquals( + ImmutableSet.of( + "blob-key-2" + ), + processor.currentKeys + ); + + } + + @Test + public void testOldValues() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Iterable mockList = ImmutableList.of( + buildMockBlob("blob-bucket-1", "blob-key-1", 2L) + ); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.enqueue("test2"); + runner.run(2); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS); + runner.assertTransferCount(ListGCSBucket.REL_SUCCESS, 1); + + assertEquals( + "blob-key-1", + runner.getStateManager().getState(Scope.CLUSTER).get(ListGCSBucket.CURRENT_KEY_PREFIX+"0") + ); + + assertEquals( + "2", + runner.getStateManager().getState(Scope.CLUSTER).get(ListGCSBucket.CURRENT_TIMESTAMP) + ); + + } + + + + @Test + public void testEmptyList() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Iterable mockList = ImmutableList.of(); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + runner.assertTransferCount(ListGCSBucket.REL_SUCCESS, 0); + + assertEquals( + "No state should be persisted on an empty return", + -1L, + runner.getStateManager().getState(Scope.CLUSTER).getVersion() + ); + } + + @Test + public void testAttributesSet() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = buildMockBlob("test-bucket-1", "test-key-1", 2L); + when(blob.getSize()).thenReturn(SIZE); + when(blob.getCacheControl()).thenReturn(CACHE_CONTROL); + when(blob.getComponentCount()).thenReturn(COMPONENT_COUNT); + when(blob.getContentEncoding()).thenReturn(CONTENT_ENCODING); + when(blob.getContentLanguage()).thenReturn(CONTENT_LANGUAGE); + when(blob.getContentType()).thenReturn(CONTENT_TYPE); + when(blob.getCrc32c()).thenReturn(CRC32C); + + final BlobInfo.CustomerEncryption mockEncryption = mock(BlobInfo.CustomerEncryption.class); + when(mockEncryption.getEncryptionAlgorithm()).thenReturn(ENCRYPTION); + when(mockEncryption.getKeySha256()).thenReturn(ENCRYPTION_SHA256); + when(blob.getCustomerEncryption()).thenReturn(mockEncryption); + + when(blob.getEtag()).thenReturn(ETAG); + when(blob.getGeneratedId()).thenReturn(GENERATED_ID); + when(blob.getGeneration()).thenReturn(GENERATION); + when(blob.getMd5()).thenReturn(MD5); + when(blob.getMediaLink()).thenReturn(MEDIA_LINK); + when(blob.getMetageneration()).thenReturn(METAGENERATION); + when(blob.getSelfLink()).thenReturn(URI); + when(blob.getContentDisposition()).thenReturn(CONTENT_DISPOSITION); + when(blob.getCreateTime()).thenReturn(CREATE_TIME); + when(blob.getUpdateTime()).thenReturn(UPDATE_TIME); + + final Iterable mockList = ImmutableList.of(blob); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + assertEquals( + CACHE_CONTROL, + flowFile.getAttribute(CACHE_CONTROL_ATTR) + ); + + assertEquals( + COMPONENT_COUNT, + Integer.valueOf(flowFile.getAttribute(COMPONENT_COUNT_ATTR)) + ); + + assertEquals( + CONTENT_ENCODING, + flowFile.getAttribute(CONTENT_ENCODING_ATTR) + ); + + assertEquals( + CONTENT_LANGUAGE, + flowFile.getAttribute(CONTENT_LANGUAGE_ATTR) + ); + + assertEquals( + CONTENT_TYPE, + flowFile.getAttribute(CoreAttributes.MIME_TYPE.key()) + ); + + assertEquals( + CRC32C, + flowFile.getAttribute(CRC32C_ATTR) + ); + + assertEquals( + ENCRYPTION, + flowFile.getAttribute(ENCRYPTION_ALGORITHM_ATTR) + ); + + assertEquals( + ENCRYPTION_SHA256, + flowFile.getAttribute(ENCRYPTION_SHA256_ATTR) + ); + + assertEquals( + ETAG, + flowFile.getAttribute(ETAG_ATTR) + ); + + assertEquals( + GENERATED_ID, + flowFile.getAttribute(GENERATED_ID_ATTR) + ); + + assertEquals( + GENERATION, + Long.valueOf(flowFile.getAttribute(GENERATION_ATTR)) + ); + + assertEquals( + MD5, + flowFile.getAttribute(MD5_ATTR) + ); + + assertEquals( + MEDIA_LINK, + flowFile.getAttribute(MEDIA_LINK_ATTR) + ); + + assertEquals( + METAGENERATION, + Long.valueOf(flowFile.getAttribute(METAGENERATION_ATTR)) + ); + + assertEquals( + URI, + flowFile.getAttribute(URI_ATTR) + ); + + assertEquals( + CONTENT_DISPOSITION, + flowFile.getAttribute(CONTENT_DISPOSITION_ATTR) + ); + + assertEquals( + CREATE_TIME, + Long.valueOf(flowFile.getAttribute(CREATE_TIME_ATTR)) + ); + + assertEquals( + UPDATE_TIME, + Long.valueOf(flowFile.getAttribute(UPDATE_TIME_ATTR)) + ); + } + + @Test + public void testAclOwnerUser() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = buildMockBlob("test-bucket-1", "test-key-1", 2L); + final Acl.User mockUser = mock(Acl.User.class); + when(mockUser.getEmail()).thenReturn(OWNER_USER_EMAIL); + when(blob.getOwner()).thenReturn(mockUser); + + final Iterable mockList = ImmutableList.of(blob); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + assertEquals( + OWNER_USER_EMAIL, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "user", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + + } + + + @Test + public void testAclOwnerGroup() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = buildMockBlob("test-bucket-1", "test-key-1", 2L); + final Acl.Group mockGroup = mock(Acl.Group.class); + when(mockGroup.getEmail()).thenReturn(OWNER_GROUP_EMAIL); + when(blob.getOwner()).thenReturn(mockGroup); + + final Iterable mockList = ImmutableList.of(blob); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + assertEquals( + OWNER_GROUP_EMAIL, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "group", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + + } + + + + @Test + public void testAclOwnerDomain() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = buildMockBlob("test-bucket-1", "test-key-1", 2L); + final Acl.Domain mockDomain = mock(Acl.Domain.class); + when(mockDomain.getDomain()).thenReturn(OWNER_DOMAIN); + when(blob.getOwner()).thenReturn(mockDomain); + + final Iterable mockList = ImmutableList.of(blob); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + assertEquals( + OWNER_DOMAIN, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "domain", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + + } + + + + @Test + public void testAclOwnerProject() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Blob blob = buildMockBlob("test-bucket-1", "test-key-1", 2L); + final Acl.Project mockProject = mock(Acl.Project.class); + when(mockProject.getProjectId()).thenReturn(OWNER_PROJECT_ID); + when(blob.getOwner()).thenReturn(mockProject); + + final Iterable mockList = ImmutableList.of(blob); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + + runner.assertAllFlowFilesTransferred(FetchGCSObject.REL_SUCCESS); + runner.assertTransferCount(FetchGCSObject.REL_SUCCESS, 1); + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(FetchGCSObject.REL_SUCCESS).get(0); + assertEquals( + OWNER_PROJECT_ID, + flowFile.getAttribute(OWNER_ATTR) + ); + + assertEquals( + "project", + flowFile.getAttribute(OWNER_TYPE_ATTR) + ); + + } + + + @Test + public void testYieldOnBadStateRestore() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + final Iterable mockList = ImmutableList.of(); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), any(Storage.BlobListOption[].class))) + .thenReturn(mockBlobPages); + + runner.getStateManager().setFailOnStateGet(Scope.CLUSTER, true); + runner.enqueue("test"); + runner.run(); + + runner.assertTransferCount(ListGCSBucket.REL_SUCCESS, 0); + assertEquals( + 1, + runner.getLogger().getErrorMessages().size() + ); + } + + @Test + public void testListOptionsPrefix() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + + runner.setProperty( + ListGCSBucket.PREFIX, + PREFIX + ); + + runner.assertValid(); + + final Iterable mockList = ImmutableList.of(); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), argumentCaptor.capture())) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + assertEquals( + Storage.BlobListOption.prefix(PREFIX), + argumentCaptor.getValue() + ); + + } + + + @Test + public void testListOptionsVersions() throws Exception { + reset(storage, mockBlobPages); + final ListGCSBucket processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + + runner.setProperty( + ListGCSBucket.USE_GENERATIONS, + String.valueOf(USE_GENERATIONS) + ); + runner.assertValid(); + + final Iterable mockList = ImmutableList.of(); + + when(mockBlobPages.getValues()) + .thenReturn(mockList); + + when(mockBlobPages.getNextPage()).thenReturn(null); + + when(storage.list(anyString(), argumentCaptor.capture())) + .thenReturn(mockBlobPages); + + runner.enqueue("test"); + runner.run(); + + Storage.BlobListOption option = argumentCaptor.getValue(); + + assertEquals( + Storage.BlobListOption.versions(true), + option + ); + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/PutGCSObjectIT.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/PutGCSObjectIT.java new file mode 100644 index 000000000000..9c1401377ca3 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/PutGCSObjectIT.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.processors.gcp.storage; + +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobId; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.junit.Test; + +import java.util.Map; + +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.junit.Assert.assertTrue; + +/** + * Integration tests for {@link PutGCSObject} which actually use Google Cloud resources. + */ +public class PutGCSObjectIT extends AbstractGCSIT { + private static final String KEY = "delete-me"; + private static final byte[] CONTENT = {12, 13, 14}; + + @Test + public void testSimplePut() throws Exception { + final TestRunner runner = buildNewRunner(new PutGCSObject()); + runner.setProperty(PutGCSObject.BUCKET, BUCKET); + runner.setProperty(PutGCSObject.KEY, KEY); + + runner.enqueue(CONTENT); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 1); + assertTrue(fileEquals(KEY, CONTENT)); + + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS).get(0); + flowFile.assertAttributeNotExists(ENCRYPTION_ALGORITHM_ATTR); + + for (Map.Entry entry : flowFile.getAttributes().entrySet()) { + System.out.println(entry.getKey() + ":" + entry.getValue()); + } + } + + @Test + public void testEncryptedPut() throws Exception { + final TestRunner runner = buildNewRunner(new PutGCSObject()); + runner.setProperty(PutGCSObject.BUCKET, BUCKET); + runner.setProperty(PutGCSObject.KEY, KEY); + runner.setProperty(PutGCSObject.ENCRYPTION_KEY, ENCRYPTION_KEY); + + runner.enqueue(CONTENT); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 1); + assertTrue(fileEqualsEncrypted(KEY, CONTENT)); + + final MockFlowFile flowFile = runner.getFlowFilesForRelationship(ListGCSBucket.REL_SUCCESS).get(0); + flowFile.assertAttributeExists(ENCRYPTION_ALGORITHM_ATTR); + + for (Map.Entry entry : flowFile.getAttributes().entrySet()) { + System.out.println(entry.getKey() + ":" + entry.getValue()); + } + } + + @Test + public void testPutWithAcl() throws Exception { + final TestRunner runner = buildNewRunner(new PutGCSObject()); + runner.setProperty(PutGCSObject.BUCKET, BUCKET); + runner.setProperty(PutGCSObject.KEY, KEY); + runner.setProperty(PutGCSObject.ACL, PutGCSObject.ACL_BUCKET_OWNER_READ); + + runner.enqueue(CONTENT); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 1); + assertTrue(fileEquals(KEY, CONTENT)); + + final Blob blob = storage.get(BlobId.of(BUCKET, KEY)); + + boolean userIsOwner = false; + boolean projectOwnerIsReader = false; + for (Acl acl : blob.listAcls()) { + if (acl.getEntity().getType() == Acl.Entity.Type.USER + && acl.getRole() == Acl.Role.OWNER) { + userIsOwner = true; + } + + if (acl.getEntity().getType() == Acl.Entity.Type.PROJECT + && acl.getRole() == Acl.Role.READER) { + projectOwnerIsReader = true; + } + } + + assertTrue(userIsOwner); + assertTrue(projectOwnerIsReader); + } + + @Test + public void testPutWithOverwrite() throws Exception { + final TestRunner runner = buildNewRunner(new PutGCSObject()); + runner.setProperty(PutGCSObject.BUCKET, BUCKET); + runner.setProperty(PutGCSObject.KEY, KEY); + + putTestFile(KEY, new byte[]{1, 2}); + + runner.enqueue(CONTENT); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_SUCCESS, 1); + assertTrue(fileEquals(KEY, CONTENT)); + + } + + + @Test + public void testPutWithNoOverwrite() throws Exception { + final TestRunner runner = buildNewRunner(new PutGCSObject()); + runner.setProperty(PutGCSObject.BUCKET, BUCKET); + runner.setProperty(PutGCSObject.KEY, KEY); + runner.setProperty(PutGCSObject.OVERWRITE, "false"); + + putTestFile(KEY, new byte[]{1, 2}); + + runner.enqueue(CONTENT); + + runner.run(); + + runner.assertAllFlowFilesTransferred(ListGCSBucket.REL_FAILURE, 1); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/PutGCSObjectTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/PutGCSObjectTest.java new file mode 100644 index 000000000000..4b6fb9abb550 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/PutGCSObjectTest.java @@ -0,0 +1,536 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF 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.gcp.storage; + +import com.google.cloud.storage.Acl; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.BlobInfo; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageException; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.nifi.flowfile.attributes.CoreAttributes; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.junit.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; + +import java.io.InputStream; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.google.cloud.storage.Storage.PredefinedAcl.BUCKET_OWNER_READ; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.BUCKET_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CACHE_CONTROL_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.COMPONENT_COUNT_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_DISPOSITION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_ENCODING_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CONTENT_LANGUAGE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CRC32C_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.CREATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_ALGORITHM_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ENCRYPTION_SHA256_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.ETAG_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATED_ID_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.GENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.KEY_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MD5_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.MEDIA_LINK_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.METAGENERATION_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.OWNER_TYPE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.SIZE_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.UPDATE_TIME_ATTR; +import static org.apache.nifi.processors.gcp.storage.StorageAttributes.URI_ATTR; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.when; + +/** + * Unit tests for {@link PutGCSObject} which do not use Google Cloud resources. + */ +public class PutGCSObjectTest extends AbstractGCSTest { + private static final String FILENAME = "test-filename"; + private static final String KEY = "test-key"; + private static final String CONTENT_TYPE = "test-content-type"; + private static final String MD5 = "test-md5"; + private static final String CRC32C = "test-crc32c"; + private static final Storage.PredefinedAcl ACL = BUCKET_OWNER_READ; + private static final String ENCRYPTION_KEY = "test-encryption-key"; + private static final Boolean OVERWRITE = false; + private static final String CONTENT_DISPOSITION_TYPE = "inline"; + + + private static final Long SIZE = 100L; + private static final String CACHE_CONTROL = "test-cache-control"; + private static final Integer COMPONENT_COUNT = 3; + private static final String CONTENT_ENCODING = "test-content-encoding"; + private static final String CONTENT_LANGUAGE = "test-content-language"; + private static final String ENCRYPTION = "test-encryption"; + private static final String ENCRYPTION_SHA256 = "test-encryption-256"; + private static final String ETAG = "test-etag"; + private static final String GENERATED_ID = "test-generated-id"; + private static final String MEDIA_LINK = "test-media-link"; + private static final Long METAGENERATION = 42L; + private static final String OWNER_USER_EMAIL = "test-owner-user-email"; + private static final String OWNER_GROUP_EMAIL = "test-owner-group-email"; + private static final String OWNER_DOMAIN = "test-owner-domain"; + private static final String OWNER_PROJECT_ID = "test-owner-project-id"; + private static final String URI = "test-uri"; + private static final String CONTENT_DISPOSITION = "attachment; filename=\"" + FILENAME + "\""; + private static final Long CREATE_TIME = 1234L; + private static final Long UPDATE_TIME = 4567L; + private final static Long GENERATION = 5L; + + @Mock + Storage storage; + + @Mock + Blob blob; + + @Captor + ArgumentCaptor blobWriteOptionArgumentCaptor; + + @Captor + ArgumentCaptor inputStreamArgumentCaptor; + + @Captor + ArgumentCaptor blobInfoArgumentCaptor; + + @Override + public PutGCSObject getProcessor() { + return new PutGCSObject() { + @Override + protected Storage getCloudService() { + return storage; + } + }; + } + + @Override + protected void addRequiredPropertiesToRunner(TestRunner runner) { + runner.setProperty(PutGCSObject.BUCKET, BUCKET); + } + + @Test + public void testSuccessfulPutOperationNoParameters() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + + runner.assertValid(); + + when(storage.create(blobInfoArgumentCaptor.capture(), + inputStreamArgumentCaptor.capture(), + blobWriteOptionArgumentCaptor.capture())).thenReturn(blob); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + /** Can't do this any more due to the switch to Java InputStreams which close after an operation **/ + /* + String text; + try (final Reader reader = new InputStreamReader(inputStreamArgumentCaptor.getValue())) { + text = CharStreams.toString(reader); + } + + assertEquals( + "FlowFile content should be equal to the Blob content", + "test", + text + ); + */ + + final List blobWriteOptions = blobWriteOptionArgumentCaptor.getAllValues(); + assertEquals("No BlobWriteOptions should be set", + 0, + blobWriteOptions.size()); + + final BlobInfo blobInfo = blobInfoArgumentCaptor.getValue(); + assertNull(blobInfo.getMd5()); + assertNull(blobInfo.getContentDisposition()); + assertNull(blobInfo.getCrc32c()); + } + + @Test + public void testSuccessfulPutOperation() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + + runner.setProperty(PutGCSObject.KEY, KEY); + runner.setProperty(PutGCSObject.CONTENT_TYPE, CONTENT_TYPE); + runner.setProperty(PutGCSObject.MD5, MD5); + runner.setProperty(PutGCSObject.CRC32C, CRC32C); + runner.setProperty(PutGCSObject.ACL, ACL.name()); + runner.setProperty(PutGCSObject.ENCRYPTION_KEY, ENCRYPTION_KEY); + runner.setProperty(PutGCSObject.OVERWRITE, String.valueOf(OVERWRITE)); + runner.setProperty(PutGCSObject.CONTENT_DISPOSITION_TYPE, CONTENT_DISPOSITION_TYPE); + + runner.assertValid(); + + when(storage.create(blobInfoArgumentCaptor.capture(), + inputStreamArgumentCaptor.capture(), + blobWriteOptionArgumentCaptor.capture())).thenReturn(blob); + + runner.enqueue("test", ImmutableMap.of(CoreAttributes.FILENAME.key(), FILENAME)); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + /* + + String text; + try (final Reader reader = new InputStreamReader(inputStreamArgumentCaptor.getValue())) { + text = CharStreams.toString(reader); + } + + assertEquals( + "FlowFile content should be equal to the Blob content", + "test", + text + ); + + */ + + final BlobInfo blobInfo = blobInfoArgumentCaptor.getValue(); + assertEquals( + BUCKET, + blobInfo.getBucket() + ); + + assertEquals( + KEY, + blobInfo.getName() + ); + + assertEquals( + CONTENT_DISPOSITION_TYPE + "; filename=" + FILENAME, + blobInfo.getContentDisposition() + ); + + assertEquals( + CONTENT_TYPE, + blobInfo.getContentType() + ); + + assertEquals( + MD5, + blobInfo.getMd5() + ); + + assertEquals( + CRC32C, + blobInfo.getCrc32c() + ); + + assertNull(blobInfo.getMetadata()); + + final List blobWriteOptions = blobWriteOptionArgumentCaptor.getAllValues(); + final Set blobWriteOptionSet = ImmutableSet.copyOf(blobWriteOptions); + + assertEquals( + "Each of the BlobWriteOptions should be unique", + blobWriteOptions.size(), + blobWriteOptionSet.size() + ); + + assertTrue("The doesNotExist BlobWriteOption should be set if OVERWRITE is false", + blobWriteOptionSet.contains(Storage.BlobWriteOption.doesNotExist())); + assertTrue("The md5Match BlobWriteOption should be set if MD5 is non-null", + blobWriteOptionSet.contains(Storage.BlobWriteOption.md5Match())); + assertTrue("The crc32cMatch BlobWriteOption should be set if CRC32C is non-null", + blobWriteOptionSet.contains(Storage.BlobWriteOption.crc32cMatch())); + assertTrue("The predefinedAcl BlobWriteOption should be set if ACL is non-null", + blobWriteOptionSet.contains(Storage.BlobWriteOption.predefinedAcl(ACL))); + assertTrue("The encryptionKey BlobWriteOption should be set if ENCRYPTION_KEY is non-null", + blobWriteOptionSet.contains(Storage.BlobWriteOption.encryptionKey(ENCRYPTION_KEY))); + + } + + @Test + public void testSuccessfulPutOperationWithUserMetadata() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + + runner.setProperty( + "testMetadataKey1", "testMetadataValue1" + ); + runner.setProperty( + "testMetadataKey2", "testMetadataValue2" + ); + + runner.assertValid(); + + when(storage.create(blobInfoArgumentCaptor.capture(), + inputStreamArgumentCaptor.capture(), + blobWriteOptionArgumentCaptor.capture())).thenReturn(blob); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + + /* + String text; + try (final Reader reader = new InputStreamReader(inputStreamArgumentCaptor.getValue())) { + text = CharStreams.toString(reader); + } + + assertEquals( + "FlowFile content should be equal to the Blob content", + "test", + text + ); + + */ + + final BlobInfo blobInfo = blobInfoArgumentCaptor.getValue(); + final Map metadata = blobInfo.getMetadata(); + + assertNotNull(metadata); + + assertEquals( + 2, + metadata.size() + ); + + assertEquals( + "testMetadataValue1", + metadata.get("testMetadataKey1") + ); + + assertEquals( + "testMetadataValue2", + metadata.get("testMetadataKey2") + ); + } + + @Test + public void testAttributesSetOnSuccessfulPut() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.create(any(BlobInfo.class), any(InputStream.class), any(Storage.BlobWriteOption.class))) + .thenReturn(blob); + + when(blob.getBucket()).thenReturn(BUCKET); + when(blob.getName()).thenReturn(KEY); + when(blob.getSize()).thenReturn(SIZE); + when(blob.getCacheControl()).thenReturn(CACHE_CONTROL); + when(blob.getComponentCount()).thenReturn(COMPONENT_COUNT); + when(blob.getContentDisposition()).thenReturn(CONTENT_DISPOSITION); + when(blob.getContentEncoding()).thenReturn(CONTENT_ENCODING); + when(blob.getContentLanguage()).thenReturn(CONTENT_LANGUAGE); + when(blob.getContentType()).thenReturn(CONTENT_TYPE); + when(blob.getCrc32c()).thenReturn(CRC32C); + + final BlobInfo.CustomerEncryption mockEncryption = mock(BlobInfo.CustomerEncryption.class); + when(blob.getCustomerEncryption()).thenReturn(mockEncryption); + when(mockEncryption.getEncryptionAlgorithm()).thenReturn(ENCRYPTION); + when(mockEncryption.getKeySha256()).thenReturn(ENCRYPTION_SHA256); + when(blob.getEtag()).thenReturn(ETAG); + when(blob.getGeneratedId()).thenReturn(GENERATED_ID); + when(blob.getGeneration()).thenReturn(GENERATION); + when(blob.getMd5()).thenReturn(MD5); + when(blob.getMediaLink()).thenReturn(MEDIA_LINK); + when(blob.getMetageneration()).thenReturn(METAGENERATION); + when(blob.getSelfLink()).thenReturn(URI); + when(blob.getCreateTime()).thenReturn(CREATE_TIME); + when(blob.getUpdateTime()).thenReturn(UPDATE_TIME); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + final MockFlowFile mockFlowFile = runner.getFlowFilesForRelationship(PutGCSObject.REL_SUCCESS).get(0); + + mockFlowFile.assertAttributeEquals(BUCKET_ATTR, BUCKET); + mockFlowFile.assertAttributeEquals(KEY_ATTR, KEY); + mockFlowFile.assertAttributeEquals(SIZE_ATTR, String.valueOf(SIZE)); + mockFlowFile.assertAttributeEquals(CACHE_CONTROL_ATTR, CACHE_CONTROL); + mockFlowFile.assertAttributeEquals(COMPONENT_COUNT_ATTR, String.valueOf(COMPONENT_COUNT)); + mockFlowFile.assertAttributeEquals(CONTENT_DISPOSITION_ATTR, CONTENT_DISPOSITION); + mockFlowFile.assertAttributeEquals(CoreAttributes.FILENAME.key(), FILENAME); + mockFlowFile.assertAttributeEquals(CONTENT_ENCODING_ATTR, CONTENT_ENCODING); + mockFlowFile.assertAttributeEquals(CONTENT_LANGUAGE_ATTR, CONTENT_LANGUAGE); + mockFlowFile.assertAttributeEquals(CoreAttributes.MIME_TYPE.key(), CONTENT_TYPE); + mockFlowFile.assertAttributeEquals(CRC32C_ATTR, CRC32C); + mockFlowFile.assertAttributeEquals(ENCRYPTION_ALGORITHM_ATTR, ENCRYPTION); + mockFlowFile.assertAttributeEquals(ENCRYPTION_SHA256_ATTR, ENCRYPTION_SHA256); + mockFlowFile.assertAttributeEquals(ETAG_ATTR, ETAG); + mockFlowFile.assertAttributeEquals(GENERATED_ID_ATTR, GENERATED_ID); + mockFlowFile.assertAttributeEquals(GENERATION_ATTR, String.valueOf(GENERATION)); + mockFlowFile.assertAttributeEquals(MD5_ATTR, MD5); + mockFlowFile.assertAttributeEquals(MEDIA_LINK_ATTR, MEDIA_LINK); + mockFlowFile.assertAttributeEquals(METAGENERATION_ATTR, String.valueOf(METAGENERATION)); + mockFlowFile.assertAttributeEquals(URI_ATTR, URI); + mockFlowFile.assertAttributeEquals(CREATE_TIME_ATTR, String.valueOf(CREATE_TIME)); + mockFlowFile.assertAttributeEquals(UPDATE_TIME_ATTR, String.valueOf(UPDATE_TIME)); + } + + @Test + public void testAclAttributeUser() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.create(any(BlobInfo.class), any(InputStream.class), any(Storage.BlobWriteOption.class))) + .thenReturn(blob); + + final Acl.User mockUser = mock(Acl.User.class); + when(mockUser.getEmail()).thenReturn(OWNER_USER_EMAIL); + when(blob.getOwner()).thenReturn(mockUser); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + final MockFlowFile mockFlowFile = runner.getFlowFilesForRelationship(PutGCSObject.REL_SUCCESS).get(0); + mockFlowFile.assertAttributeEquals(OWNER_ATTR, OWNER_USER_EMAIL); + mockFlowFile.assertAttributeEquals(OWNER_TYPE_ATTR, "user"); + } + + @Test + public void testAclAttributeGroup() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.create(any(BlobInfo.class), any(InputStream.class), any(Storage.BlobWriteOption.class))) + .thenReturn(blob); + + final Acl.Group mockGroup = mock(Acl.Group.class); + when(mockGroup.getEmail()).thenReturn(OWNER_GROUP_EMAIL); + when(blob.getOwner()).thenReturn(mockGroup); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + final MockFlowFile mockFlowFile = runner.getFlowFilesForRelationship(PutGCSObject.REL_SUCCESS).get(0); + mockFlowFile.assertAttributeEquals(OWNER_ATTR, OWNER_GROUP_EMAIL); + mockFlowFile.assertAttributeEquals(OWNER_TYPE_ATTR, "group"); + } + + + @Test + public void testAclAttributeDomain() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.create(any(BlobInfo.class), any(InputStream.class), any(Storage.BlobWriteOption.class))) + .thenReturn(blob); + + final Acl.Domain mockDomain = mock(Acl.Domain.class); + when(mockDomain.getDomain()).thenReturn(OWNER_DOMAIN); + when(blob.getOwner()).thenReturn(mockDomain); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + final MockFlowFile mockFlowFile = runner.getFlowFilesForRelationship(PutGCSObject.REL_SUCCESS).get(0); + mockFlowFile.assertAttributeEquals(OWNER_ATTR, OWNER_DOMAIN); + mockFlowFile.assertAttributeEquals(OWNER_TYPE_ATTR, "domain"); + } + + + @Test + public void testAclAttributeProject() throws Exception { + reset(storage, blob); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.create(any(BlobInfo.class), any(InputStream.class), any(Storage.BlobWriteOption.class))) + .thenReturn(blob); + + final Acl.Project mockProject = mock(Acl.Project.class); + when(mockProject.getProjectId()).thenReturn(OWNER_PROJECT_ID); + when(blob.getOwner()).thenReturn(mockProject); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_SUCCESS); + runner.assertTransferCount(PutGCSObject.REL_SUCCESS, 1); + + final MockFlowFile mockFlowFile = runner.getFlowFilesForRelationship(PutGCSObject.REL_SUCCESS).get(0); + mockFlowFile.assertAttributeEquals(OWNER_ATTR, OWNER_PROJECT_ID); + mockFlowFile.assertAttributeEquals(OWNER_TYPE_ATTR, "project"); + } + + @Test + public void testFailureHandling() throws Exception { + reset(storage); + final PutGCSObject processor = getProcessor(); + final TestRunner runner = buildNewRunner(processor); + addRequiredPropertiesToRunner(runner); + runner.assertValid(); + + when(storage.create(any(BlobInfo.class), any(InputStream.class), any(Storage.BlobWriteOption.class))) + .thenThrow(new StorageException(404, "test exception")); + + runner.enqueue("test"); + runner.run(); + + runner.assertAllFlowFilesTransferred(PutGCSObject.REL_FAILURE); + runner.assertTransferCount(PutGCSObject.REL_FAILURE, 1); + + final MockFlowFile mockFlowFile = runner.getFlowFilesForRelationship(PutGCSObject.REL_FAILURE).get(0); + assertTrue(mockFlowFile.isPenalized()); + } + +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/StorageAttributesTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/StorageAttributesTest.java new file mode 100644 index 000000000000..61ec9f50d082 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/StorageAttributesTest.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.processors.gcp.storage; + +import org.junit.Test; + +import java.lang.reflect.Constructor; +import java.lang.reflect.Modifier; + +import static org.junit.Assert.assertTrue; + +public class StorageAttributesTest { + @Test + public void testStorageAttributeClassCannotBeInvoked() throws Exception { + Constructor constructor = StorageAttributes.class.getDeclaredConstructor(); + assertTrue("Constructor of StorageAttributes should be private", Modifier.isPrivate(constructor.getModifiers())); + constructor.setAccessible(true); + constructor.newInstance(); + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/UtilTest.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/UtilTest.java new file mode 100644 index 000000000000..0b2287d2d929 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/storage/UtilTest.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.processors.gcp.storage; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +/** + * Tests the Util class static methods. + */ +public class UtilTest { + @Test + public void testContentDispositionParsing() throws Exception { + final String contentDisposition = "attachment; filename=\"plans.pdf\""; + + final Util.ParsedContentDisposition parsed = Util.parseContentDisposition(contentDisposition); + assertNotNull(parsed); + assertEquals("plans.pdf", + parsed.getFileName()); + + assertEquals("attachment", + parsed.getContentDispositionType()); + } + + @Test + public void testContentDispositionParsingBadParse() throws Exception { + final String contentDisposition = "bad-header"; + + assertNull(Util.parseContentDisposition(contentDisposition)); + } +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mock-gcp-application-default-credentials.json b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mock-gcp-application-default-credentials.json new file mode 100644 index 000000000000..f045ca69ffbf --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mock-gcp-application-default-credentials.json @@ -0,0 +1,6 @@ +{ + "client_id": "123456789012-af23m23321knfg00ekrjlwke90rjkewl.apps.googleusercontent.com", + "client_secret": "d-MbDD42LcmsVNVdls21dBAs", + "refresh_token": "1/cvZBer532GBbzsxdf7jj7LOvd-IcmbSa5tgVcls5j5z", + "type": "authorized_user" +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mock-gcp-service-account.json b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mock-gcp-service-account.json new file mode 100644 index 000000000000..c1d9190e14b2 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mock-gcp-service-account.json @@ -0,0 +1,12 @@ +{ + "type": "service_account", + "project_id": "test", + "private_key_id": "testHash", + "private_key": "-----BEGIN PRIVATE KEY-----\nMIIEvgIBADANBgkqhkiG9w0BAQEFAASCBKgwggSkAgEAAoIBAQDBewuX0bvBzOq0\nWu0AGbJHvDiCim1jJ/Oec7AIKW2IlafMY5VYussKVyt7zI1GIPrOyMmVed9eTd0E\niAahfYhgsVUEl9fd4m53Gm9A+I14C7p0PLC5IByc+DDYKpHtAf9CPggHcZExiDqU\n0NyxzEcimi/o9Qev2MJH3tv8jNzOVNfv1B5K6DYrce++x+8gU/RetXWn9htwIAsE\nHdfyKR+J7RR8/tUtqx/19RuXMTKCpBKDcHa5LFpv6jA0yLjp+4QeQcCa63azCoJx\nhmD4Y+gvxRV5nGxk7QIAWiax4A5PNlJcYBAn188isC1ZgDU+kyyM+nrejXVoip5r\nd3b/A7sjAgMBAAECggEBALTqeoemzRtFon2svAoo/QSI4opmKCzcsbeLU6H+Ivbh\ngXrj70V9vNfZdMaZGczmj7+GDsDfqdcDldRj4VdmC3zmtKnL1kUbMtHZ/QfSom4L\nAXkpOtKQTVEV3o5zF+p3wJjPajCTqAGZ8bUvq/3xFt8rL/t0C5EJbXlI0YlQqjOf\nj7y/XEo+EBBCwNazkX45tPK0EwQuo8XsWBeCX59BGR5An/LIOixdvxRMaprApNpH\ndS3Ap3d9s0c4YXgugpBbbiVL2hnqag4teXmX6p0tZbDone4VrpSJWOhSnvuBpHbi\neVyu3ByhsgRKKwm0UnOOFvM5N1kUNjIs+TQ4n/Cv4ukCgYEA5Q2+3frLip1AXbXv\n/FuL5WXBgNH9dHIoYVvERu7e7sjUwpqbQn0va+biYzqCoR3UzUAgPPi8U8YdQN1Y\nKJz6dQGDeKvPwVuBr2+s3Va9s8L52tV0dXA0sHb8aFxfc2E+zPQH3eWCRuoWcwZr\nx3YLXvNVnSwDvJB/+q3EhNRwYu0CgYEA2D34bBtqcwZDcgmPdKsPVoshk1uEsa8d\n90i725c6XVbKs5TP/cOWT+DsJmEfHF/mWneJZt0aTh6O6tpEDtIIISBvqdNoztnv\nt4tQ9+HD3p5JjoFeIZlFaTQQZoCC1PgfYa9xutxO8hltaWpp50+S00wEdgUA0zCM\nj884Vhk/hE8CgYA3Ub6LNgr6i0gEWfB/7kw3NwAo8I5aFUgTW2poB0DoQrC/3z8o\nK7vMP5LljDgIWYAPojEnCJvTT8G47Lxh8qe6oobyGeyvMj579Gi3fD+MrsZRR8Q8\nqMDQ7avAOK8E2rOkJDvSJ5/zKI4Lcb2OCsBsSjCfKQYuAGgoTtdrjTMncQKBgAUH\nS+OXr54FI0RfnIpl//FPQvSeSDOpktTRSC0PEzhgcE5Ew6FvDuvEmzk5QPPz9vNb\nnEJcGeR/KWukr7h4gd/jVTVpySImR0DJaJSbF2bx31wE/h9h5Q9ROqBnlKNHMdOf\ntNFXli5jEPxGkTfjzdJEDkaAT0iZ9GrTssetxqBZAoGBAISdPHJSot5rgT5ILqQk\nYVjLopguymhz1QzyXoe1g9lC9KQIUQo6iikmocPpWiupktaB5Ck7gmwsMnWYS2ti\nDeWDAS+QC5W3wy40Gos4SN/FsZKTHD87SPHY82rx0/GvbXJKqZmYMM6M6+fM/jJd\n+kaA70VDxYg60IdOgf7o9HqA\n-----END PRIVATE KEY-----", + "client_email": "test@developer.gserviceaccount.com", + "client_id": "123456789", + "auth_uri": "https://accounts.google.com/o/oauth2/auth", + "token_uri": "https://accounts.google.com/o/oauth2/token", + "auth_provider_x509_cert_url": "https://www.googleapis.com/oauth2/v1/certs", + "client_x509_cert_url": "https://www.googleapis.com/robot/v1/metadata/x509/test%40developer.gserviceaccount.com" +} diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 000000000000..ca6ee9cea8ec --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1 @@ +mock-maker-inline \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-gcp-bundle/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/pom.xml new file mode 100644 index 000000000000..7e413dfd5830 --- /dev/null +++ b/nifi-nar-bundles/nifi-gcp-bundle/pom.xml @@ -0,0 +1,34 @@ + + + + 4.0.0 + + + org.apache.nifi + nifi-nar-bundles + 1.2.0-SNAPSHOT + + + nifi-gcp-bundle + 1.2.0-SNAPSHOT + pom + + + nifi-gcp-processors + nifi-gcp-nar + + + diff --git a/nifi-nar-bundles/pom.xml b/nifi-nar-bundles/pom.xml index df3aa7d1d4ad..f0164aff6fe8 100755 --- a/nifi-nar-bundles/pom.xml +++ b/nifi-nar-bundles/pom.xml @@ -73,6 +73,7 @@ nifi-ranger-bundle nifi-websocket-bundle nifi-tcp-bundle + nifi-gcp-bundle diff --git a/pom.xml b/pom.xml index 0fb3d1bac3bf..f1de76b8d2db 100644 --- a/pom.xml +++ b/pom.xml @@ -1287,6 +1287,12 @@ language governing permissions and limitations under the License. --> 1.2.0-SNAPSHOT nar + + org.apache.nifi + nifi-gcp-nar + 1.2.0-SNAPSHOT + nar + org.apache.nifi nifi-properties