You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/07/12 20:30:40 UTC

[GitHub] [nifi] tpalfy opened a new pull request, #6200: NIFI-10223 Created ListGoogleDrive processor.

tpalfy opened a new pull request, #6200:
URL: https://github.com/apache/nifi/pull/6200

   <!-- Licensed to the Apache Software Foundation (ASF) under one or more -->
   <!-- contributor license agreements.  See the NOTICE file distributed with -->
   <!-- this work for additional information regarding copyright ownership. -->
   <!-- The ASF licenses this file to You under the Apache License, Version 2.0 -->
   <!-- (the "License"); you may not use this file except in compliance with -->
   <!-- the License.  You may obtain a copy of the License at -->
   <!--     http://www.apache.org/licenses/LICENSE-2.0 -->
   <!-- Unless required by applicable law or agreed to in writing, software -->
   <!-- distributed under the License is distributed on an "AS IS" BASIS, -->
   <!-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -->
   <!-- See the License for the specific language governing permissions and -->
   <!-- limitations under the License. -->
   
   # Summary
   
   [NIFI-10223](https://issues.apache.org/jira/browse/NIFI-10223)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r921087293


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml:
##########
@@ -114,6 +114,11 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>com.google.apis</groupId>
+            <artifactId>google-api-services-drive</artifactId>
+            <version>v3-rev20211107-1.32.1</version>
+        </dependency>

Review Comment:
   Maven Central includes newer revisions of the driver, so it would be helpful to use the latest version unless there is a reason for using this older revision.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] arpadboda commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
arpadboda commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r919984798


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/util/GoogleUtils.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.gcp.util;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+
+public class GoogleUtils {
+    /**
+     * 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")

Review Comment:
   Maybe I'm wrong, but as far as I see the builder only supports one name. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] arpadboda commented on pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
arpadboda commented on PR #6200:
URL: https://github.com/apache/nifi/pull/6200#issuecomment-1189050835

   Nope, thanks, I'm good, approved!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] tpalfy commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r924483200


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile." +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file." +
+                " The reason for this is that the original modified date of a file is preserved when uploaded to Google Drive." +
+                " 'Created time' takea the time when the upload occurs. However uploaded files can still be modified later."),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})

Review Comment:
   I see that in the majority of cases the simple descriptions don't have a period at the end. I'd rather leave them that way.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on PR #6200:
URL: https://github.com/apache/nifi/pull/6200#issuecomment-1190163862

   Thanks everyone!
   Merging to main...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r925159212


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml:
##########
@@ -153,6 +158,7 @@
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
                     <excludes combine.children="append">
+                        <exclude>src/test/resources/ListGoogleDriveIT/test_file.txt</exclude>

Review Comment:
   This line can be removed



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] asfgit closed pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #6200: NIFI-10223 Created ListGoogleDrive processor.
URL: https://github.com/apache/nifi/pull/6200


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r920881589


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/AbstractGCPProcessor.java:
##########
@@ -105,17 +106,6 @@
             .sensitive(true)
             .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();
-

Review Comment:
   In order to keep backward compatibility, this `public` field should not be removed (3rd party custom processors may use it).
   Workaround could be:
   ```
   @Deprecated
   public static final PropertyDescriptor GCP_CREDENTIALS_PROVIDER_SERVICE = GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE;
   ```



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml:
##########
@@ -114,6 +114,11 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>com.google.apis</groupId>
+            <artifactId>google-api-services-drive</artifactId>
+            <version>v3-rev20211107-1.32.1</version>
+        </dependency>

Review Comment:
   Please check the implications of this new dependency in terms of licensing. I think it needs to be added in the NOTICE file.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r924501341


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/drive/ListGoogleDriveIT.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport;
+import com.google.api.client.http.FileContent;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors;
+import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsControllerService;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * CREDENTIAL_JSON_FILE_PATH - A Service Account credentials JSON file.<br />
+ * SHARED_FOLDER_ID - The ID of a Folder that is shared with the Service Account. The test will create files and sub-folders within this folder.<br />
+ * <br />
+ * Created files and folders are cleaned up, but it's advisable to dedicate a folder for this test so that it can be cleaned up easily should the test fail to do so.
+ * <br /><br />
+ * WARNING: The creation of a file is not a synchronized operation so tests may fail because the processor may not list all of them.
+ */
+public class ListGoogleDriveIT {
+    public static final String CREDENTIAL_JSON_FILE_PATH = "";
+    public static final String SHARED_FOLDER_ID = "";
+
+    public static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private TestRunner testRunner;
+
+    private Drive driveService;
+
+    private String mainFolderId;
+
+    @BeforeEach
+    public void init() throws Exception {
+        ListGoogleDrive testSubject = new ListGoogleDrive();
+
+        testRunner = TestRunners.newTestRunner(testSubject);
+
+        GCPCredentialsControllerService gcpCredentialsControllerService = new GCPCredentialsControllerService();
+        testRunner.addControllerService("gcp_credentials_provider_service", gcpCredentialsControllerService);
+        testRunner.setProperty(gcpCredentialsControllerService, CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE, CREDENTIAL_JSON_FILE_PATH);
+        testRunner.enableControllerService(gcpCredentialsControllerService);
+        testRunner.setProperty(GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE, "gcp_credentials_provider_service");
+
+        NetHttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport();
+
+        driveService = new Drive.Builder(
+                httpTransport,
+                JSON_FACTORY,
+                testSubject.getHttpCredentialsAdapter(
+                        testRunner.getProcessContext(),
+                        DriveScopes.all()
+                )
+        )
+                .setApplicationName(this.getClass().getSimpleName())
+                .build();
+
+        File file = createFolder("main", SHARED_FOLDER_ID);
+
+        mainFolderId = file.getId();
+        testRunner.setProperty(ListGoogleDrive.FOLDER_ID, mainFolderId);
+    }
+
+    @AfterEach
+    void tearDown() throws IOException {
+        driveService.files()
+                .delete(mainFolderId)
+                .execute();
+    }
+
+    @Test
+    void listFilesFrom3LayerDeepDirectoryTree() throws Exception {
+        // GIVEN
+        File main_sub1 = createFolder("main_sub1", mainFolderId);
+        File main_sub2 = createFolder("main_sub2", mainFolderId);
+
+        File main_sub1_sub1 = createFolder("main_sub1_sub1", main_sub1.getId());
+
+        createFile("main_file1", mainFolderId);
+        createFile("main_file2", mainFolderId);
+        createFile("main_file3", mainFolderId);
+
+        createFile("main_sub1_file1", main_sub1.getId());
+
+        createFile("main_sub2_file1", main_sub2.getId());
+        createFile("main_sub2_file2", main_sub2.getId());
+
+        createFile("main_sub1_sub1_file1", main_sub1_sub1.getId());
+        createFile("main_sub1_sub1_file2", main_sub1_sub1.getId());
+        createFile("main_sub1_sub1_file3", main_sub1_sub1.getId());
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_file1", "main_file2", "main_file3",
+                "main_sub1_file1",
+                "main_sub2_file1", "main_sub2_file2",
+                "main_sub1_sub1_file1", "main_sub1_sub1_file2", "main_sub1_sub1_file3"
+        ));
+
+        // The creation of the files are not (completely) synchronized.
+        Thread.sleep(2000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+
+        // Next, list a sub folder, non-recursively this time. (Changing these properties will clear the Processor state as well
+        //  so all files are eligible for listing again.)
+
+        // GIVEN
+        testRunner.clearTransferState();
+
+        expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_sub1_file1"
+        ));
+
+        // WHEN
+        testRunner.setProperty(ListGoogleDrive.FOLDER_ID, main_sub1.getId());
+        testRunner.setProperty(ListGoogleDrive.RECURSIVE_SEARCH, "false");
+        testRunner.run();
+
+        // THEN
+        successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    @Test
+    void doNotListTooYoungFilesWhenMinAgeIsSet() throws Exception {
+        // GIVEN
+        testRunner.setProperty(ListGoogleDrive.MIN_AGE, "15 s");
+
+        createFile("main_file", mainFolderId);
+
+        // Make sure the file 'arrives' and could be listed
+        Thread.sleep(5000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(Collections.emptySet(), actualFileNames);
+
+        // Next, wait for another 10+ seconds for MIN_AGE to expire then list again
+
+        // GIVEN
+        Thread.sleep(10000);
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_file"
+        ));
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    private File createFolder(String folderName, String... parentFolderIds) throws IOException {
+        File fileMetaData = new File();
+        fileMetaData.setName(folderName);
+
+        if (parentFolderIds != null) {
+            fileMetaData.setParents(Arrays.asList(parentFolderIds));
+        }
+
+        fileMetaData.setMimeType("application/vnd.google-apps.folder");
+
+        Drive.Files.Create create = driveService.files()
+                .create(fileMetaData)
+                .setFields("id");
+
+        File file = create.execute();
+
+        return file;
+    }
+
+    private File createFile(String name, String... folderIds) throws IOException {
+        File fileMetadata = new File();
+        fileMetadata.setName(name);
+        fileMetadata.setParents(Arrays.asList(folderIds));
+
+        FileContent content = new FileContent("text/plain", new java.io.File("src/test/resources/ListGoogleDriveIT/test_file.txt"));

Review Comment:
   Instead of checking in this file to version control, it would be better to create a temporary file that would be deleted after test completion.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile. " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file." +
+                " The reason for this is that the original modified date of a file is preserved when uploaded to Google Drive." +
+                " 'Created time' takes the time when the upload occurs. However uploaded files can still be modified later."),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "The processor stores necessary data to be able to keep track what files have been listed already." +
+        " What exactly needs to be stored depends on the 'Listing Strategy'." +
+        " 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID of the folder from which to pull list of files. Needs to be shared with a Service Account." +
+                    " WARNING: Unauthorized access to the folder is treated as if the folder was empty." +
+                    " This results in the processor not creating result flowfiles. No additional error message is provided.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("When 'true', will include list of files from concrete sub-folders (ignores shortcuts)." +
+                    " Otherwise, will return only files that have the defined 'Folder ID' as their parent directly." +
+                    " WARNING: The listing may fail if there are too many sub-folders (500+).")
+            .required(true)
+            .defaultValue("true")
+            .allowableValues("true", "false")
+            .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+            .name("min-age")
+            .displayName("Minimum File Age")
+            .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored.")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
+
+    public static final PropertyDescriptor LISTING_STRATEGY = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractListProcessor.LISTING_STRATEGY)
+            .allowableValues(BY_TIMESTAMPS, BY_ENTITIES, BY_TIME_WINDOW, NO_TRACKING)
+            .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE,
+            FOLDER_ID,
+            RECURSIVE_SEARCH,
+            MIN_AGE,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET,
+            RECORD_WRITER,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxyAwareTransportFactory.PROXY_SPECS)
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext validationContext, Collection<ValidationResult> results) {
+    }
+
+    @Override
+    protected Map<String, String> createAttributes(
+            final GoogleDriveFileInfo entity,
+            final ProcessContext context
+    ) {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("drive.id", entity.getId());
+        attributes.put("filename", entity.getName());
+        attributes.put("drive.size", String.valueOf(entity.getSize()));
+        attributes.put("drive.timestamp", String.valueOf(entity.getTimestamp()));
+        attributes.put("mime.type", entity.getMimeType());
+
+        return attributes;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws IOException {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        httpTransport = new ProxyAwareTransportFactory(proxyConfiguration).create();
+    }
+
+    @Override
+    protected String getListingContainerName(final ProcessContext context) {
+        return String.format("Google Drive Folder [%s]", getPath(context));
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(final PropertyDescriptor property) {
+        return LISTING_STRATEGY.equals(property)
+                || FOLDER_ID.equals(property)
+                || RECURSIVE_SEARCH.equals(property);
+    }
+
+    @Override
+    protected Scope getStateScope(final PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return GoogleDriveFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected List<GoogleDriveFileInfo> performListing(
+            final ProcessContext context,
+            final Long minTimestamp,
+            final ListingMode listingMode
+    ) throws IOException {
+        final List<GoogleDriveFileInfo> listing = new ArrayList<>();
+
+        final String folderId = context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+        final Boolean recursive = context.getProperty(RECURSIVE_SEARCH).asBoolean();
+        final Long minAge = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        Drive driveService = new Drive.Builder(
+                this.httpTransport,
+                JSON_FACTORY,
+                getHttpCredentialsAdapter(
+                        context,
+                        Arrays.asList(DriveScopes.DRIVE_METADATA_READONLY)
+                )
+        )
+                .setApplicationName(APPLICATION_NAME)
+                .build();
+
+        StringBuilder queryBuilder = new StringBuilder();
+        queryBuilder.append(buildQueryForDirs(driveService, folderId, recursive));
+        queryBuilder.append(" and (mimeType != 'application/vnd.google-apps.folder')");
+        queryBuilder.append(" and (mimeType != 'application/vnd.google-apps.shortcut')");
+        queryBuilder.append(" and trashed = false");
+        if (minTimestamp != null && minTimestamp > 0) {
+            String formattedMinTimestamp = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(OffsetDateTime.ofInstant(Instant.ofEpochMilli(minTimestamp), ZoneId.of("UTC")));

Review Comment:
   It looks like `ZoneId.of("UTC")` could be replaced with `ZoneOffset.UTC`.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml:
##########
@@ -114,6 +114,11 @@
                 </exclusion>
             </exclusions>
         </dependency>
+        <dependency>
+            <groupId>com.google.apis</groupId>
+            <artifactId>google-api-services-drive</artifactId>
+            <version>v3-rev20220508-1.32.1</version>

Review Comment:
   The latest version appears to be [v3-rev20220709-1.32.1](https://search.maven.org/artifact/com.google.apis/google-api-services-drive/v3-rev20220709-1.32.1/jar), can this be updated?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r923026765


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file"),

Review Comment:
   The user can set the file's timestamp to a date in the future locally. I would consider to use the created date only which is assigned by Google Drive and seems to be consistent.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +

Review Comment:
   There is also another type of usage: use the Record Writer property and add the listing as a json in the FlowFile content. It could be mentioned in the documentation.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file"),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored. " +

Review Comment:
   Storing the timestamp is only applicable to the Timestamp Tracking strategy. It could be highlighted in the documentation.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file"),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored. " +
+        "This allows the Processor to list only files 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID of the folder from which to pull list of files. Needs to be shared with a Service Account.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("When 'true', will include list of files from sub-folders. Otherwise, will return only files that have the defined 'Folder ID' as their parent directly.")
+            .required(true)
+            .defaultValue("true")
+            .allowableValues("true", "false")
+            .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+            .name("min-age")
+            .displayName("Minimum File Age")
+            .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE,
+            FOLDER_ID,
+            RECURSIVE_SEARCH,
+            MIN_AGE,
+            LISTING_STRATEGY,
+            RECORD_WRITER,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxyAwareTransportFactory.PROXY_SPECS)
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext validationContext, Collection<ValidationResult> results) {
+    }
+
+    @Override
+    protected Map<String, String> createAttributes(
+            final GoogleDriveFileInfo entity,
+            final ProcessContext context
+    ) {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("drive.id", entity.getId());
+        attributes.put("filename", entity.getName());
+        attributes.put("drive.size", String.valueOf(entity.getSize()));
+        attributes.put("drive.timestamp", String.valueOf(entity.getTimestamp()));
+        attributes.put("mime.type", entity.getMimeType());
+
+        return attributes;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws IOException {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        httpTransport = new ProxyAwareTransportFactory(proxyConfiguration).create();
+    }
+
+    @Override
+    protected String getListingContainerName(final ProcessContext context) {
+        return String.format("Google Drive Folder [%s]", getPath(context));
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(final PropertyDescriptor property) {
+        return LISTING_STRATEGY.equals(property)
+                || FOLDER_ID.equals(property)
+                || RECURSIVE_SEARCH.equals(property);
+    }
+
+    @Override
+    protected Scope getStateScope(final PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return GoogleDriveFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected List<GoogleDriveFileInfo> performListing(
+            final ProcessContext context,
+            final Long minTimestamp,
+            final ListingMode listingMode
+    ) throws IOException {
+        final List<GoogleDriveFileInfo> listing = new ArrayList<>();
+
+        final String folderId = context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+        final Boolean recursive = context.getProperty(RECURSIVE_SEARCH).asBoolean();
+        final Long minAge = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        Drive driveService = new Drive.Builder(
+                this.httpTransport,
+                JSON_FACTORY,
+                getHttpCredentialsAdapter(
+                        context,
+                        Arrays.asList(DriveScopes.DRIVE_METADATA_READONLY)
+                )
+        )
+                .setApplicationName(APPLICATION_NAME)
+                .build();
+
+        StringBuilder queryBuilder = new StringBuilder();
+        queryBuilder.append(buildQueryForDirs(driveService, folderId, recursive));
+        queryBuilder.append(" and (mimeType != 'application/vnd.google-apps.folder')");
+        queryBuilder.append(" and (mimeType != 'application/vnd.google-apps.shortcut')");

Review Comment:
   Please document it for the user that shortcuts will not be followed.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,351 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file"),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored. " +
+        "This allows the Processor to list only files 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID of the folder from which to pull list of files. Needs to be shared with a Service Account.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("When 'true', will include list of files from sub-folders. Otherwise, will return only files that have the defined 'Folder ID' as their parent directly.")
+            .required(true)
+            .defaultValue("true")
+            .allowableValues("true", "false")
+            .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+            .name("min-age")
+            .displayName("Minimum File Age")
+            .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE,
+            FOLDER_ID,
+            RECURSIVE_SEARCH,
+            MIN_AGE,
+            LISTING_STRATEGY,
+            RECORD_WRITER,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET,
+            ProxyConfiguration.createProxyConfigPropertyDescriptor(false, ProxyAwareTransportFactory.PROXY_SPECS)
+    ));
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext validationContext, Collection<ValidationResult> results) {
+    }
+
+    @Override
+    protected Map<String, String> createAttributes(
+            final GoogleDriveFileInfo entity,
+            final ProcessContext context
+    ) {
+        final Map<String, String> attributes = new HashMap<>();
+        attributes.put("drive.id", entity.getId());
+        attributes.put("filename", entity.getName());
+        attributes.put("drive.size", String.valueOf(entity.getSize()));
+        attributes.put("drive.timestamp", String.valueOf(entity.getTimestamp()));
+        attributes.put("mime.type", entity.getMimeType());
+
+        return attributes;
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) throws IOException {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context);
+        httpTransport = new ProxyAwareTransportFactory(proxyConfiguration).create();
+    }
+
+    @Override
+    protected String getListingContainerName(final ProcessContext context) {
+        return String.format("Google Drive Folder [%s]", getPath(context));
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(final PropertyDescriptor property) {
+        return LISTING_STRATEGY.equals(property)
+                || FOLDER_ID.equals(property)
+                || RECURSIVE_SEARCH.equals(property);
+    }
+
+    @Override
+    protected Scope getStateScope(final PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return GoogleDriveFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected List<GoogleDriveFileInfo> performListing(
+            final ProcessContext context,
+            final Long minTimestamp,
+            final ListingMode listingMode
+    ) throws IOException {
+        final List<GoogleDriveFileInfo> listing = new ArrayList<>();
+
+        final String folderId = context.getProperty(FOLDER_ID).evaluateAttributeExpressions().getValue();
+        final Boolean recursive = context.getProperty(RECURSIVE_SEARCH).asBoolean();
+        final Long minAge = context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        Drive driveService = new Drive.Builder(
+                this.httpTransport,
+                JSON_FACTORY,
+                getHttpCredentialsAdapter(
+                        context,
+                        Arrays.asList(DriveScopes.DRIVE_METADATA_READONLY)
+                )
+        )
+                .setApplicationName(APPLICATION_NAME)
+                .build();
+
+        StringBuilder queryBuilder = new StringBuilder();
+        queryBuilder.append(buildQueryForDirs(driveService, folderId, recursive));
+        queryBuilder.append(" and (mimeType != 'application/vnd.google-apps.folder')");
+        queryBuilder.append(" and (mimeType != 'application/vnd.google-apps.shortcut')");
+        queryBuilder.append(" and trashed = false");
+        if (minTimestamp != null && minTimestamp > 0) {
+            String formattedMinTimestamp = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(OffsetDateTime.ofInstant(Instant.ofEpochMilli(minTimestamp), ZoneId.of("UTC")));
+
+            queryBuilder.append(" and (");
+            queryBuilder.append("modifiedTime >= '" + formattedMinTimestamp + "'");
+            queryBuilder.append(" or createdTime >= '" + formattedMinTimestamp + "'");
+            queryBuilder.append(")");
+        }
+        if (minAge != null && minAge > 0) {
+            long maxTimestamp = System.currentTimeMillis() - minAge;
+            String formattedMaxTimestamp = DateTimeFormatter.ISO_OFFSET_DATE_TIME.format(OffsetDateTime.ofInstant(Instant.ofEpochMilli(maxTimestamp), ZoneId.of("UTC")));
+
+            queryBuilder.append(" and modifiedTime < '" + formattedMaxTimestamp + "'");
+            queryBuilder.append(" and createdTime < '" + formattedMaxTimestamp + "'");
+        }
+
+        String pageToken = null;
+        do {
+            FileList result = driveService.files()
+                    .list()
+                    .setQ(queryBuilder.toString())

Review Comment:
   There seems to be a limit for the length of the query string: 30000 characters. It means we can add 500-550 folders in this query.
   It should be documented for the user that they cannot list Google Drive shares with more than 500 subfolders.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] arpadboda commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
arpadboda commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r919974450


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time of the file"),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored. " +
+        "This allows the Processor to list only files 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID on the folder from which to pull list of files. Needs to be shared with a Service Account.")

Review Comment:
   typo: the ID *of* the folder 



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists files in a Google Drive folder. Listing details are attached to an empty FlowFile for use with FetchGoogleDrive.  " +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time of the file"),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "After performing a listing of files, the timestamp of the newest file is stored. " +
+        "This allows the Processor to list only files 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID on the folder from which to pull list of files. Needs to be shared with a Service Account.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("When 'true', will pull list of files from sub-folders. Otherwise, will return only files that have the defined 'Folder ID' as their parent directly.")

Review Comment:
   Nitpicking: will include list of files from subfolders



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r924822788


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/drive/ListGoogleDriveIT.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport;
+import com.google.api.client.http.FileContent;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors;
+import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsControllerService;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * CREDENTIAL_JSON_FILE_PATH - A Service Account credentials JSON file.<br />
+ * SHARED_FOLDER_ID - The ID of a Folder that is shared with the Service Account. The test will create files and sub-folders within this folder.<br />
+ * <br />
+ * Created files and folders are cleaned up, but it's advisable to dedicate a folder for this test so that it can be cleaned up easily should the test fail to do so.
+ * <br /><br />
+ * WARNING: The creation of a file is not a synchronized operation so tests may fail because the processor may not list all of them.
+ */
+public class ListGoogleDriveIT {
+    public static final String CREDENTIAL_JSON_FILE_PATH = "";
+    public static final String SHARED_FOLDER_ID = "";
+
+    public static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private TestRunner testRunner;
+
+    private Drive driveService;
+
+    private String mainFolderId;
+
+    @BeforeEach
+    public void init() throws Exception {
+        ListGoogleDrive testSubject = new ListGoogleDrive();
+
+        testRunner = TestRunners.newTestRunner(testSubject);
+
+        GCPCredentialsControllerService gcpCredentialsControllerService = new GCPCredentialsControllerService();
+        testRunner.addControllerService("gcp_credentials_provider_service", gcpCredentialsControllerService);
+        testRunner.setProperty(gcpCredentialsControllerService, CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE, CREDENTIAL_JSON_FILE_PATH);
+        testRunner.enableControllerService(gcpCredentialsControllerService);
+        testRunner.setProperty(GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE, "gcp_credentials_provider_service");
+
+        NetHttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport();
+
+        driveService = new Drive.Builder(
+                httpTransport,
+                JSON_FACTORY,
+                testSubject.getHttpCredentialsAdapter(
+                        testRunner.getProcessContext(),
+                        DriveScopes.all()
+                )
+        )
+                .setApplicationName(this.getClass().getSimpleName())
+                .build();
+
+        File file = createFolder("main", SHARED_FOLDER_ID);
+
+        mainFolderId = file.getId();
+        testRunner.setProperty(ListGoogleDrive.FOLDER_ID, mainFolderId);
+    }
+
+    @AfterEach
+    void tearDown() throws IOException {
+        driveService.files()
+                .delete(mainFolderId)
+                .execute();
+    }
+
+    @Test
+    void listFilesFrom3LayerDeepDirectoryTree() throws Exception {
+        // GIVEN
+        File main_sub1 = createFolder("main_sub1", mainFolderId);
+        File main_sub2 = createFolder("main_sub2", mainFolderId);
+
+        File main_sub1_sub1 = createFolder("main_sub1_sub1", main_sub1.getId());
+
+        createFile("main_file1", mainFolderId);
+        createFile("main_file2", mainFolderId);
+        createFile("main_file3", mainFolderId);
+
+        createFile("main_sub1_file1", main_sub1.getId());
+
+        createFile("main_sub2_file1", main_sub2.getId());
+        createFile("main_sub2_file2", main_sub2.getId());
+
+        createFile("main_sub1_sub1_file1", main_sub1_sub1.getId());
+        createFile("main_sub1_sub1_file2", main_sub1_sub1.getId());
+        createFile("main_sub1_sub1_file3", main_sub1_sub1.getId());
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_file1", "main_file2", "main_file3",
+                "main_sub1_file1",
+                "main_sub2_file1", "main_sub2_file2",
+                "main_sub1_sub1_file1", "main_sub1_sub1_file2", "main_sub1_sub1_file3"
+        ));
+
+        // The creation of the files are not (completely) synchronized.
+        Thread.sleep(2000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+
+        // Next, list a sub folder, non-recursively this time. (Changing these properties will clear the Processor state as well
+        //  so all files are eligible for listing again.)
+
+        // GIVEN
+        testRunner.clearTransferState();
+
+        expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_sub1_file1"
+        ));
+
+        // WHEN
+        testRunner.setProperty(ListGoogleDrive.FOLDER_ID, main_sub1.getId());
+        testRunner.setProperty(ListGoogleDrive.RECURSIVE_SEARCH, "false");
+        testRunner.run();
+
+        // THEN
+        successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    @Test
+    void doNotListTooYoungFilesWhenMinAgeIsSet() throws Exception {
+        // GIVEN
+        testRunner.setProperty(ListGoogleDrive.MIN_AGE, "15 s");
+
+        createFile("main_file", mainFolderId);
+
+        // Make sure the file 'arrives' and could be listed
+        Thread.sleep(5000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(Collections.emptySet(), actualFileNames);
+
+        // Next, wait for another 10+ seconds for MIN_AGE to expire then list again
+
+        // GIVEN
+        Thread.sleep(10000);
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_file"
+        ));
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    private File createFolder(String folderName, String... parentFolderIds) throws IOException {
+        File fileMetaData = new File();
+        fileMetaData.setName(folderName);
+
+        if (parentFolderIds != null) {
+            fileMetaData.setParents(Arrays.asList(parentFolderIds));
+        }
+
+        fileMetaData.setMimeType("application/vnd.google-apps.folder");
+
+        Drive.Files.Create create = driveService.files()
+                .create(fileMetaData)
+                .setFields("id");
+
+        File file = create.execute();
+
+        return file;
+    }
+
+    private File createFile(String name, String... folderIds) throws IOException {
+        File fileMetadata = new File();
+        fileMetadata.setName(name);
+        fileMetadata.setParents(Arrays.asList(folderIds));
+
+        FileContent content = new FileContent("text/plain", new java.io.File("src/test/resources/ListGoogleDriveIT/test_file.txt"));

Review Comment:
   @exceptionfactory @tpalfy What do you think about `ByteArrayContent`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r924860751


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/test/java/org/apache/nifi/processors/gcp/drive/ListGoogleDriveIT.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport;
+import com.google.api.client.http.FileContent;
+import com.google.api.client.http.javanet.NetHttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import org.apache.nifi.processors.gcp.credentials.factory.CredentialPropertyDescriptors;
+import org.apache.nifi.processors.gcp.credentials.service.GCPCredentialsControllerService;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * Set the following constants before running:<br />
+ * <br />
+ * CREDENTIAL_JSON_FILE_PATH - A Service Account credentials JSON file.<br />
+ * SHARED_FOLDER_ID - The ID of a Folder that is shared with the Service Account. The test will create files and sub-folders within this folder.<br />
+ * <br />
+ * Created files and folders are cleaned up, but it's advisable to dedicate a folder for this test so that it can be cleaned up easily should the test fail to do so.
+ * <br /><br />
+ * WARNING: The creation of a file is not a synchronized operation so tests may fail because the processor may not list all of them.
+ */
+public class ListGoogleDriveIT {
+    public static final String CREDENTIAL_JSON_FILE_PATH = "";
+    public static final String SHARED_FOLDER_ID = "";
+
+    public static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private TestRunner testRunner;
+
+    private Drive driveService;
+
+    private String mainFolderId;
+
+    @BeforeEach
+    public void init() throws Exception {
+        ListGoogleDrive testSubject = new ListGoogleDrive();
+
+        testRunner = TestRunners.newTestRunner(testSubject);
+
+        GCPCredentialsControllerService gcpCredentialsControllerService = new GCPCredentialsControllerService();
+        testRunner.addControllerService("gcp_credentials_provider_service", gcpCredentialsControllerService);
+        testRunner.setProperty(gcpCredentialsControllerService, CredentialPropertyDescriptors.SERVICE_ACCOUNT_JSON_FILE, CREDENTIAL_JSON_FILE_PATH);
+        testRunner.enableControllerService(gcpCredentialsControllerService);
+        testRunner.setProperty(GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE, "gcp_credentials_provider_service");
+
+        NetHttpTransport httpTransport = GoogleNetHttpTransport.newTrustedTransport();
+
+        driveService = new Drive.Builder(
+                httpTransport,
+                JSON_FACTORY,
+                testSubject.getHttpCredentialsAdapter(
+                        testRunner.getProcessContext(),
+                        DriveScopes.all()
+                )
+        )
+                .setApplicationName(this.getClass().getSimpleName())
+                .build();
+
+        File file = createFolder("main", SHARED_FOLDER_ID);
+
+        mainFolderId = file.getId();
+        testRunner.setProperty(ListGoogleDrive.FOLDER_ID, mainFolderId);
+    }
+
+    @AfterEach
+    void tearDown() throws IOException {
+        driveService.files()
+                .delete(mainFolderId)
+                .execute();
+    }
+
+    @Test
+    void listFilesFrom3LayerDeepDirectoryTree() throws Exception {
+        // GIVEN
+        File main_sub1 = createFolder("main_sub1", mainFolderId);
+        File main_sub2 = createFolder("main_sub2", mainFolderId);
+
+        File main_sub1_sub1 = createFolder("main_sub1_sub1", main_sub1.getId());
+
+        createFile("main_file1", mainFolderId);
+        createFile("main_file2", mainFolderId);
+        createFile("main_file3", mainFolderId);
+
+        createFile("main_sub1_file1", main_sub1.getId());
+
+        createFile("main_sub2_file1", main_sub2.getId());
+        createFile("main_sub2_file2", main_sub2.getId());
+
+        createFile("main_sub1_sub1_file1", main_sub1_sub1.getId());
+        createFile("main_sub1_sub1_file2", main_sub1_sub1.getId());
+        createFile("main_sub1_sub1_file3", main_sub1_sub1.getId());
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_file1", "main_file2", "main_file3",
+                "main_sub1_file1",
+                "main_sub2_file1", "main_sub2_file2",
+                "main_sub1_sub1_file1", "main_sub1_sub1_file2", "main_sub1_sub1_file3"
+        ));
+
+        // The creation of the files are not (completely) synchronized.
+        Thread.sleep(2000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+
+        // Next, list a sub folder, non-recursively this time. (Changing these properties will clear the Processor state as well
+        //  so all files are eligible for listing again.)
+
+        // GIVEN
+        testRunner.clearTransferState();
+
+        expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_sub1_file1"
+        ));
+
+        // WHEN
+        testRunner.setProperty(ListGoogleDrive.FOLDER_ID, main_sub1.getId());
+        testRunner.setProperty(ListGoogleDrive.RECURSIVE_SEARCH, "false");
+        testRunner.run();
+
+        // THEN
+        successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    @Test
+    void doNotListTooYoungFilesWhenMinAgeIsSet() throws Exception {
+        // GIVEN
+        testRunner.setProperty(ListGoogleDrive.MIN_AGE, "15 s");
+
+        createFile("main_file", mainFolderId);
+
+        // Make sure the file 'arrives' and could be listed
+        Thread.sleep(5000);
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        List<MockFlowFile> successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        Set<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(Collections.emptySet(), actualFileNames);
+
+        // Next, wait for another 10+ seconds for MIN_AGE to expire then list again
+
+        // GIVEN
+        Thread.sleep(10000);
+
+        Set<String> expectedFileNames = new HashSet<>(Arrays.asList(
+                "main_file"
+        ));
+
+        // WHEN
+        testRunner.run();
+
+        // THEN
+        successFlowFiles = testRunner.getFlowFilesForRelationship(ListGoogleDrive.REL_SUCCESS);
+
+        actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(Collectors.toSet());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    private File createFolder(String folderName, String... parentFolderIds) throws IOException {
+        File fileMetaData = new File();
+        fileMetaData.setName(folderName);
+
+        if (parentFolderIds != null) {
+            fileMetaData.setParents(Arrays.asList(parentFolderIds));
+        }
+
+        fileMetaData.setMimeType("application/vnd.google-apps.folder");
+
+        Drive.Files.Create create = driveService.files()
+                .create(fileMetaData)
+                .setFields("id");
+
+        File file = create.execute();
+
+        return file;
+    }
+
+    private File createFile(String name, String... folderIds) throws IOException {
+        File fileMetadata = new File();
+        fileMetadata.setName(name);
+        fileMetadata.setParents(Arrays.asList(folderIds));
+
+        FileContent content = new FileContent("text/plain", new java.io.File("src/test/resources/ListGoogleDriveIT/test_file.txt"));

Review Comment:
   Good point @turcsanyip, using `ByteArrayContent` seems like a better solution than a local file path.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on a diff in pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on code in PR #6200:
URL: https://github.com/apache/nifi/pull/6200#discussion_r923680558


##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile." +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file." +
+                " The reason for this is that the original modified date of a file is preserved when uploaded to Google Drive." +
+                " 'Created time' takea the time when the upload occurs. However uploaded files can still be modified later."),

Review Comment:
   Typo: takes



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile." +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file." +
+                " The reason for this is that the original modified date of a file is preserved when uploaded to Google Drive." +
+                " 'Created time' takea the time when the upload occurs. However uploaded files can still be modified later."),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "The processor stores necessary data to be able to keep track what files have been listed already." +
+        " What exactly needs to be stored depends on the 'Listing Strategy'." +
+        " 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID of the folder from which to pull list of files. Needs to be shared with a Service Account." +
+                    " WARNING: Unauthorized access to the folder is treated as if the folder was empty." +
+                    " This results in the processor not creating result flowfiles. No additional error message is provided.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("When 'true', will include list of files from concrete sub-folders (ignores shortcuts)." +
+                    " Otherwise, will return only files that have the defined 'Folder ID' as their parent directly." +
+                    " WARNING: The listing may fail if there are too many sub-folders (500+).")
+            .required(true)
+            .defaultValue("true")
+            .allowableValues("true", "false")
+            .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+            .name("min-age")
+            .displayName("Minimum File Age")
+            .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored")

Review Comment:
   Typo: no period at the end of the sentence.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile." +
+        "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 " +

Review Comment:
   Typo: no space before "This..."



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile." +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file." +
+                " The reason for this is that the original modified date of a file is preserved when uploaded to Google Drive." +
+                " 'Created time' takea the time when the upload occurs. However uploaded files can still be modified later."),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})

Review Comment:
   Please use period at the end of all attribute descriptions. As in case of the description of `drive.timestamp`.



##########
nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/drive/ListGoogleDrive.java:
##########
@@ -0,0 +1,364 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.drive;
+
+import com.google.api.client.http.HttpTransport;
+import com.google.api.client.json.JsonFactory;
+import com.google.api.client.json.gson.GsonFactory;
+import com.google.api.services.drive.Drive;
+import com.google.api.services.drive.DriveScopes;
+import com.google.api.services.drive.model.File;
+import com.google.api.services.drive.model.FileList;
+import com.google.auth.http.HttpCredentialsAdapter;
+import com.google.auth.oauth2.GoogleCredentials;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.PrimaryNodeOnly;
+import org.apache.nifi.annotation.behavior.Stateful;
+import org.apache.nifi.annotation.behavior.TriggerSerially;
+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.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.gcp.credentials.service.GCPCredentialsService;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.processors.gcp.util.GoogleUtils;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.time.Instant;
+import java.time.OffsetDateTime;
+import java.time.ZoneId;
+import java.time.format.DateTimeFormatter;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"google", "drive", "storage"})
+@CapabilityDescription("Lists concrete files (shortcuts are ignored) in a Google Drive folder. " +
+        "Each listed file may result in one flowfile, the metadata being written as flowfile attributes. " +
+        "Or - in case the 'Record Writer' property is set - the entire result is written as records to a single flowfile." +
+        "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.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({@WritesAttribute(attribute = "drive.id", description = "The id of the file"),
+        @WritesAttribute(attribute = "filename", description = "The name of the file"),
+        @WritesAttribute(attribute = "drive.size", description = "The size of the file"),
+        @WritesAttribute(attribute = "drive.timestamp", description = "The last modified time or created time (whichever is greater) of the file." +
+                " The reason for this is that the original modified date of a file is preserved when uploaded to Google Drive." +
+                " 'Created time' takea the time when the upload occurs. However uploaded files can still be modified later."),
+        @WritesAttribute(attribute = "mime.type", description = "MimeType of the file")})
+@Stateful(scopes = {Scope.CLUSTER}, description = "The processor stores necessary data to be able to keep track what files have been listed already." +
+        " What exactly needs to be stored depends on the 'Listing Strategy'." +
+        " 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.")
+public class ListGoogleDrive extends AbstractListProcessor<GoogleDriveFileInfo> {
+    private static final String APPLICATION_NAME = "NiFi";
+    private static final JsonFactory JSON_FACTORY = GsonFactory.getDefaultInstance();
+
+    private volatile HttpTransport httpTransport;
+
+    public static final PropertyDescriptor FOLDER_ID = new PropertyDescriptor.Builder()
+            .name("folder-id")
+            .displayName("Folder ID")
+            .description("The ID of the folder from which to pull list of files. Needs to be shared with a Service Account." +
+                    " WARNING: Unauthorized access to the folder is treated as if the folder was empty." +
+                    " This results in the processor not creating result flowfiles. No additional error message is provided.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("When 'true', will include list of files from concrete sub-folders (ignores shortcuts)." +
+                    " Otherwise, will return only files that have the defined 'Folder ID' as their parent directly." +
+                    " WARNING: The listing may fail if there are too many sub-folders (500+).")
+            .required(true)
+            .defaultValue("true")
+            .allowableValues("true", "false")
+            .build();
+
+    public static final PropertyDescriptor MIN_AGE = new PropertyDescriptor.Builder()
+            .name("min-age")
+            .displayName("Minimum File Age")
+            .description("The minimum age a file must be in order to be considered; any files younger than this will be ignored")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
+
+    public static final PropertyDescriptor LISTING_STRATEGY = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(AbstractListProcessor.LISTING_STRATEGY)
+            .allowableValues(BY_TIMESTAMPS, BY_ENTITIES, BY_TIME_WINDOW, NO_TRACKING)
+            .build();
+
+    public static final PropertyDescriptor TRACKING_STATE_CACHE = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_STATE_CACHE)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor TRACKING_TIME_WINDOW = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.TRACKING_TIME_WINDOW)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    public static final PropertyDescriptor INITIAL_LISTING_TARGET = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(ListedEntityTracker.INITIAL_LISTING_TARGET)
+            .dependsOn(LISTING_STRATEGY, BY_ENTITIES)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            GoogleUtils.GCP_CREDENTIALS_PROVIDER_SERVICE,
+            FOLDER_ID,
+            RECURSIVE_SEARCH,
+            MIN_AGE,
+            LISTING_STRATEGY,
+            RECORD_WRITER,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET,

Review Comment:
   I'd suggest moving these properties up, just after the `LISTING_STRATEGY` which they are depend on.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [nifi] turcsanyip commented on pull request #6200: NIFI-10223 Created ListGoogleDrive processor.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on PR #6200:
URL: https://github.com/apache/nifi/pull/6200#issuecomment-1187091511

   @tpalfy When the credential used for the processor is valid but has no permission to the directory, then there is no error in NiFi, the processor just does not list any files.
   Not sure if it can be handled as error due to the concept of Google Drive (getting all granted files and filtering them instead of getting the required files and checking the permission). In this case, the behaviour should be documented. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org