turcsanyip commented on code in PR #6331:
URL: https://github.com/apache/nifi/pull/6331#discussion_r962514720


##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/ListDropbox.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxRequestConfig;
+import com.dropbox.core.oauth.DbxCredential;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.ListFolderBuilder;
+import com.dropbox.core.v2.files.ListFolderResult;
+import com.dropbox.core.v2.files.Metadata;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+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.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialDetails;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.serialization.record.RecordSchema;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"dropbox", "storage"})
+@CapabilityDescription("Retrieves a listing of files from Dropbox." +
+        " Each listed file may result in one FlowFile, the metadata being 
written as Flowfile attributes." +
+        " When 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 = DropboxFileInfo.ID, 
description = "The Dropbox identifier of the file"),
+        @WritesAttribute(attribute = DropboxFileInfo.PATH, description = "The 
folder path where the file is located"),
+        @WritesAttribute(attribute = DropboxFileInfo.FILENAME, description = 
"The name of the file"),
+        @WritesAttribute(attribute = DropboxFileInfo.SIZE, description = "The 
size of the file"),
+        @WritesAttribute(attribute = DropboxFileInfo.TIMESTAMP, description = 
"The server modified time, when the file was uploaded to Dropbox"),
+        @WritesAttribute(attribute = DropboxFileInfo.REVISION, description = 
"Revision 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'.")
+public class ListDropbox extends AbstractListProcessor<DropboxFileInfo>  {
+
+    public static final PropertyDescriptor FOLDER_NAME = new 
PropertyDescriptor.Builder()
+            .name("folder-name")
+            .displayName("Folder Name")
+            .description("The name of the folder from which to pull list of 
files."+
+                    " Providing empty string as folder lists files from user 
root directory." +
+                    " Folder name should match the following regular 
expression pattern: (/(.|[\\r\\n])*)?|id:.*|(ns:[0-9]+(/.*)?) ")
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            
.addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("(/(.|[\\r\\n])*)?|id:.*|(ns:[0-9]+(/.*)?)")))
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new 
PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("Indicates whether to list files from subfolders of 
the Dropbox folder")
+            .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 newer than this will be ignored.")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
+
+    public static final PropertyDescriptor CREDENTIAL_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("dropbox-credential-service")
+            .displayName("Dropbox Credential Service")
+            .description("Controller Service used to obtain Dropbox 
credentials (App Key, App Secret, Access Token, Refresh Token)."+
+                    " See controller service's usage documentation for more 
details")
+            .identifiesControllerService(DropboxCredentialService.class)
+            .required(true)
+            .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(
+            FOLDER_NAME,
+            RECURSIVE_SEARCH,
+            MIN_AGE,
+            CREDENTIAL_SERVICE,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET,
+            RECORD_WRITER
+    ));
+
+    private DbxClientV2 dropboxApiClient;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Map<String, String> createAttributes(
+            final DropboxFileInfo entity,
+            final ProcessContext context) {
+        final Map<String, String> attributes = new HashMap<>();
+
+        for (DropboxFlowFileAttribute attribute : 
DropboxFlowFileAttribute.values()) {
+            Optional.ofNullable(attribute.getValue(entity))
+                    .ifPresent(value -> attributes.put(attribute.getName(), 
value));
+        }
+
+        return attributes;
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return 
context.getProperty(FOLDER_NAME).evaluateAttributeExpressions().getValue();
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context);
+    }
+
+    protected DbxClientV2 getDropboxApiClient(ProcessContext context) {
+        final DropboxCredentialService credentialService = 
context.getProperty(CREDENTIAL_SERVICE)
+                .asControllerService(DropboxCredentialService.class);
+        DbxRequestConfig config = new DbxRequestConfig(format("%s-%s", 
getClass().getSimpleName(), getIdentifier()));
+        DropboxCredentialDetails credential = 
credentialService.getDropboxCredential();
+        return new DbxClientV2(config, new 
DbxCredential(credential.getAccessToken(), -1L,
+                credential.getRefreshToken(), credential.getAppKey(), 
credential.getAppSecret()));
+    }
+
+    @Override
+    protected List<DropboxFileInfo> performListing(ProcessContext context, 
Long minTimestamp,
+            ListingMode listingMode) throws IOException {
+        final List<DropboxFileInfo> listing = new ArrayList<>();
+
+        final String folderName = 
context.getProperty(FOLDER_NAME).evaluateAttributeExpressions().getValue();
+        final Boolean recursive = 
context.getProperty(RECURSIVE_SEARCH).asBoolean();
+        final Long minAge = 
context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        try {
+            ListFolderBuilder listFolderBuilder = 
dropboxApiClient.files().listFolderBuilder(folderName);
+            ListFolderResult result = listFolderBuilder
+                    .withRecursive(recursive)
+                    .start();
+
+            Predicate<Metadata> metadataFilter = metadata -> true;
+
+            if (minTimestamp != null && minTimestamp > 0) {
+                metadataFilter = metadataFilter.and(metadata ->
+                        ((FileMetadata) 
metadata).getServerModified().getTime()>= minTimestamp);
+            }
+
+            if (minAge != null && minAge > 0) {
+                long maxTimestamp = System.currentTimeMillis() - minAge;
+                metadataFilter = metadataFilter.and(metadata ->
+                        ((FileMetadata) 
metadata).getServerModified().getTime() < maxTimestamp);
+            }
+
+            List<Metadata> metadataList = new 
ArrayList<>(filterMetadata(result, metadataFilter));
+
+            while (result.getHasMore()) {
+                result = 
dropboxApiClient.files().listFolderContinue(result.getCursor());
+                metadataList.addAll(filterMetadata(result, metadataFilter));
+            }
+
+            for (Metadata metadata : metadataList) {
+                DropboxFileInfo.Builder builder = new DropboxFileInfo.Builder()
+                        .id(((FileMetadata) metadata).getId())
+                        .path(metadata.getPathLower())

Review Comment:
   Not sure but `getPathDisplay()` may be a better option to use because it 
would show the path as it appears on the Dropbox UI (case-sensitive).



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/test/java/org/apache/nifi/processors/dropbox/ListDropboxTest.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.dropbox;
+
+import static org.apache.nifi.util.EqualsWrapper.wrapList;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.DbxUserFilesRequests;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.ListFolderBuilder;
+import com.dropbox.core.v2.files.ListFolderResult;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.util.EqualsWrapper;
+import org.apache.nifi.util.MockPropertyValue;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class ListDropboxTest {
+
+    private ListDropbox testSubject;
+
+    @Mock
+    private ProcessContext mockProcessContext;
+    @Mock
+    private DbxClientV2 mockDbxClient;
+    @Mock
+    private DbxUserFilesRequests mockDbxUserFilesRequest;
+    @Mock
+    private ListFolderResult mockListFolderResult;
+    @Mock
+    private ListFolderBuilder mockListFolderBuilder;
+
+
+    @BeforeEach
+    void setUp() {
+        testSubject = new ListDropbox() {
+
+            @Override
+            public DbxClientV2 getDropboxApiClient(ProcessContext context) {
+                return mockDbxClient;
+            }
+        };
+        testSubject.onScheduled(mockProcessContext);
+    }
+
+    @Test
+    void testCreatedListableEntityContainsCorrectDataOldItemFiltered() throws 
Exception {
+        long minTimestamp = 1659707000;
+
+        String id1 = "id:11111";
+        String id2 = "id:22222";
+        String filename1 = "file_name_1";
+        String old_file_name = "old_file_name";

Review Comment:
   Typo: `oldFileName`



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/test/java/org/apache/nifi/processors/dropbox/ListDropboxIT.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.dropbox;
+
+import static java.util.Spliterators.spliteratorUnknownSize;
+import static java.util.stream.Collectors.toList;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Spliterator;
+import java.util.stream.StreamSupport;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ListDropboxIT extends AbstractDropboxIT<ListDropbox>{
+
+    private static final String YOUNG_FILE_NAME = "just_created" ;
+
+    @BeforeEach
+    public void init() throws Exception {
+        super.init();
+        testRunner.setProperty(ListDropbox.FOLDER_NAME, MAIN_FOLDER);
+    }
+
+    @Override
+    protected ListDropbox createTestSubject() {
+        return new ListDropbox();
+    }
+
+    @Test
+    void testEmbeddedDirectoriesAreListed() throws Exception {
+        createFile("test_file1", "test_file_content1", MAIN_FOLDER);
+        createFile("test_file2", "test_file_content2", MAIN_FOLDER);
+        createFile("test_file11", "test_file_content11", MAIN_FOLDER + 
"/testFolder1");
+        createFile("test_file112", "test_file_content112", MAIN_FOLDER + 
"/testFolder2");
+
+        createFile("test_file_not_in_main_folder", "test_file_content31", 
"/notMainFolder");
+
+        List<String> expectedFileNames = Arrays.asList("test_file1", 
"test_file2", "test_file11",
+                "test_file112");
+
+        waitForFileCreation();
+
+        testRunner.run();
+
+        List<MockFlowFile> successFlowFiles = 
testRunner.getFlowFilesForRelationship(ListDropbox.REL_SUCCESS);
+
+        List<String> actualFileNames = successFlowFiles.stream()
+                .map(flowFile -> flowFile.getAttribute("filename"))
+                .collect(toList());
+
+        assertEquals(expectedFileNames, actualFileNames);
+    }
+
+    @Test
+    void testTooYoungFilesNotListedWhenMinAgeIsSet() throws Exception {
+        testRunner.setProperty(ListDropbox.MIN_AGE, "15 s");
+
+        createFile(YOUNG_FILE_NAME, "test_file_content1", MAIN_FOLDER);
+
+        // Make sure the file 'arrives' and could be listed
+        Thread.sleep(5000);

Review Comment:
   `waitForFileCreation()` should be used instead (like in case of the other 2 
test cases).



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/test/java/org/apache/nifi/processors/dropbox/ListDropboxIT.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.dropbox;
+
+import static java.util.Spliterators.spliteratorUnknownSize;
+import static java.util.stream.Collectors.toList;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Spliterator;
+import java.util.stream.StreamSupport;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.RecordSetWriterFactory;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+public class ListDropboxIT extends AbstractDropboxIT<ListDropbox>{
+
+    private static final String YOUNG_FILE_NAME = "just_created" ;
+
+    @BeforeEach
+    public void init() throws Exception {
+        super.init();
+        testRunner.setProperty(ListDropbox.FOLDER_NAME, MAIN_FOLDER);
+    }
+
+    @Override
+    protected ListDropbox createTestSubject() {
+        return new ListDropbox();
+    }
+
+    @Test
+    void testEmbeddedDirectoriesAreListed() throws Exception {
+        createFile("test_file1", "test_file_content1", MAIN_FOLDER);
+        createFile("test_file2", "test_file_content2", MAIN_FOLDER);
+        createFile("test_file11", "test_file_content11", MAIN_FOLDER + 
"/testFolder1");
+        createFile("test_file112", "test_file_content112", MAIN_FOLDER + 
"/testFolder2");
+
+        createFile("test_file_not_in_main_folder", "test_file_content31", 
"/notMainFolder");
+
+        List<String> expectedFileNames = Arrays.asList("test_file1", 
"test_file2", "test_file11",
+                "test_file112");

Review Comment:
   Minor: The max line length is 200 so there is no need for this type of line 
breaks.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/services/dropbox/DropboxCredentialControllerService.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.services.dropbox;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.AbstractControllerService;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialDetails;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+@CapabilityDescription("Defines credentials for Dropbox processors.")
+@Tags({ "dropbox", "credentials", "provider" })
+public class DropboxCredentialControllerService extends 
AbstractControllerService implements DropboxCredentialService {
+
+    public static final PropertyDescriptor APP_KEY = new 
PropertyDescriptor.Builder()
+            .name("app-key")
+            .displayName("App Key")
+            .description("App Key of the user's Dropbox app."+
+                    " See controller service's usage documentation for more 
details about App Key")
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor APP_SECRET = new 
PropertyDescriptor.Builder()
+            .name("app-secret")
+            .displayName("App Secret")
+            .description("App Secret of the user's Dropbox app."+
+                    " See controller service's usage documentation for more 
details about App Secret")
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor ACCESS_TOKEN = new 
PropertyDescriptor.Builder()
+            .name("access-token")
+            .displayName("Access Token")
+            .description("Access Token of the user's Dropbox app." +
+                    " See controller service's usage documentation for more 
details about Access Token generation")
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor REFRESH_TOKEN = new 
PropertyDescriptor.Builder()
+            .name("refresh-token")
+            .displayName("Refresh Token")
+            .description("Refresh Token of the user's Dropbox app."+
+                    " See controller service's usage documentation for more 
details about Refresh Token generation")
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            .sensitive(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+
+    private static final List<PropertyDescriptor> properties;

Review Comment:
   Please use uppercase for constants.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/test/java/org/apache/nifi/processors/dropbox/ListDropboxTest.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.dropbox;
+
+import static org.apache.nifi.util.EqualsWrapper.wrapList;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.DbxUserFilesRequests;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.ListFolderBuilder;
+import com.dropbox.core.v2.files.ListFolderResult;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.util.EqualsWrapper;
+import org.apache.nifi.util.MockPropertyValue;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class ListDropboxTest {
+
+    private ListDropbox testSubject;
+
+    @Mock
+    private ProcessContext mockProcessContext;
+    @Mock
+    private DbxClientV2 mockDbxClient;
+    @Mock
+    private DbxUserFilesRequests mockDbxUserFilesRequest;
+    @Mock
+    private ListFolderResult mockListFolderResult;
+    @Mock
+    private ListFolderBuilder mockListFolderBuilder;
+
+
+    @BeforeEach
+    void setUp() {
+        testSubject = new ListDropbox() {
+
+            @Override
+            public DbxClientV2 getDropboxApiClient(ProcessContext context) {
+                return mockDbxClient;
+            }
+        };
+        testSubject.onScheduled(mockProcessContext);
+    }
+
+    @Test
+    void testCreatedListableEntityContainsCorrectDataOldItemFiltered() throws 
Exception {
+        long minTimestamp = 1659707000;
+
+        String id1 = "id:11111";
+        String id2 = "id:22222";
+        String filename1 = "file_name_1";
+        String old_file_name = "old_file_name";
+        long size = 125;
+        long oldCreatedTime = 1657375066;
+        long createdTime = 1659707000;
+        String revision = "5e4ddb1320676a5c29261";
+
+        boolean isRecursive = true;
+        String folderName = "test_folder";
+
+        
when(mockProcessContext.getProperty(ListDropbox.FOLDER_NAME)).thenReturn(new 
MockPropertyValue(folderName));
+        when(mockProcessContext.getProperty(ListDropbox.RECURSIVE_SEARCH))
+                .thenReturn(new 
MockPropertyValue(String.valueOf(isRecursive)));
+        
when(mockProcessContext.getProperty(ListDropbox.MIN_AGE)).thenReturn(new 
MockPropertyValue("0 sec"));
+
+        when(mockDbxClient.files()).thenReturn(mockDbxUserFilesRequest);
+        
when(mockDbxUserFilesRequest.listFolderBuilder(folderName)).thenReturn(mockListFolderBuilder);
+        
when(mockListFolderBuilder.withRecursive(isRecursive)).thenReturn(mockListFolderBuilder);
+        when(mockListFolderBuilder.start()).thenReturn(mockListFolderResult);
+
+        when(mockListFolderResult.getEntries()).thenReturn(Arrays.asList(
+                createFileMetaData(filename1, id1, createdTime, revision, 
size),
+                createFileMetaData(old_file_name, id2, oldCreatedTime, 
revision, size)
+        ));
+        when(mockListFolderResult.getHasMore()).thenReturn(false);
+
+        List<DropboxFileInfo> expected = Collections.singletonList(
+                new DropboxFileInfo.Builder()
+                        .id(id1)
+                        .name(filename1)
+                        .size(size)
+                        .timestamp(createdTime)
+                        .revision(revision)
+                        .build()
+        );
+
+        List<DropboxFileInfo> actual = 
testSubject.performListing(mockProcessContext, minTimestamp, null);
+
+        List<Function<DropboxFileInfo, Object>> propertyProviders = 
Arrays.asList(
+                DropboxFileInfo::getId,
+                DropboxFileInfo::getName,
+                DropboxFileInfo::getSize,
+                DropboxFileInfo::getTimestamp,
+                DropboxFileInfo::getRevision
+        );
+
+        List<EqualsWrapper<DropboxFileInfo>> expectedWrapper = 
wrapList(expected, propertyProviders);
+        List<EqualsWrapper<DropboxFileInfo>> actualWrapper = wrapList(actual, 
propertyProviders);

Review Comment:
   `DropboxFileInfo` has a proper `equals()` method so I don't think 
`EqualsWrapper` is needed.



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/main/java/org/apache/nifi/processors/dropbox/ListDropbox.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.dropbox;
+
+import static java.lang.String.format;
+
+import com.dropbox.core.DbxException;
+import com.dropbox.core.DbxRequestConfig;
+import com.dropbox.core.oauth.DbxCredential;
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.ListFolderBuilder;
+import com.dropbox.core.v2.files.ListFolderResult;
+import com.dropbox.core.v2.files.Metadata;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+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.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialDetails;
+import org.apache.nifi.dropbox.credentials.service.DropboxCredentialService;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+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.serialization.record.RecordSchema;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"dropbox", "storage"})
+@CapabilityDescription("Retrieves a listing of files from Dropbox." +
+        " Each listed file may result in one FlowFile, the metadata being 
written as Flowfile attributes." +
+        " When 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 = DropboxFileInfo.ID, 
description = "The Dropbox identifier of the file"),
+        @WritesAttribute(attribute = DropboxFileInfo.PATH, description = "The 
folder path where the file is located"),
+        @WritesAttribute(attribute = DropboxFileInfo.FILENAME, description = 
"The name of the file"),
+        @WritesAttribute(attribute = DropboxFileInfo.SIZE, description = "The 
size of the file"),
+        @WritesAttribute(attribute = DropboxFileInfo.TIMESTAMP, description = 
"The server modified time, when the file was uploaded to Dropbox"),
+        @WritesAttribute(attribute = DropboxFileInfo.REVISION, description = 
"Revision 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'.")
+public class ListDropbox extends AbstractListProcessor<DropboxFileInfo>  {
+
+    public static final PropertyDescriptor FOLDER_NAME = new 
PropertyDescriptor.Builder()
+            .name("folder-name")
+            .displayName("Folder Name")
+            .description("The name of the folder from which to pull list of 
files."+
+                    " Providing empty string as folder lists files from user 
root directory." +
+                    " Folder name should match the following regular 
expression pattern: (/(.|[\\r\\n])*)?|id:.*|(ns:[0-9]+(/.*)?) ")
+            
.expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+            
.addValidator(StandardValidators.createRegexMatchingValidator(Pattern.compile("(/(.|[\\r\\n])*)?|id:.*|(ns:[0-9]+(/.*)?)")))
+            .build();
+
+    public static final PropertyDescriptor RECURSIVE_SEARCH = new 
PropertyDescriptor.Builder()
+            .name("recursive-search")
+            .displayName("Search Recursively")
+            .description("Indicates whether to list files from subfolders of 
the Dropbox folder")
+            .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 newer than this will be ignored.")
+            .required(true)
+            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
+            .defaultValue("0 sec")
+            .build();
+
+    public static final PropertyDescriptor CREDENTIAL_SERVICE = new 
PropertyDescriptor.Builder()
+            .name("dropbox-credential-service")
+            .displayName("Dropbox Credential Service")
+            .description("Controller Service used to obtain Dropbox 
credentials (App Key, App Secret, Access Token, Refresh Token)."+
+                    " See controller service's usage documentation for more 
details")
+            .identifiesControllerService(DropboxCredentialService.class)
+            .required(true)
+            .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(
+            FOLDER_NAME,
+            RECURSIVE_SEARCH,
+            MIN_AGE,
+            CREDENTIAL_SERVICE,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET,
+            RECORD_WRITER
+    ));
+
+    private DbxClientV2 dropboxApiClient;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @Override
+    protected Map<String, String> createAttributes(
+            final DropboxFileInfo entity,
+            final ProcessContext context) {
+        final Map<String, String> attributes = new HashMap<>();
+
+        for (DropboxFlowFileAttribute attribute : 
DropboxFlowFileAttribute.values()) {
+            Optional.ofNullable(attribute.getValue(entity))
+                    .ifPresent(value -> attributes.put(attribute.getName(), 
value));
+        }
+
+        return attributes;
+    }
+
+    @Override
+    protected String getPath(final ProcessContext context) {
+        return 
context.getProperty(FOLDER_NAME).evaluateAttributeExpressions().getValue();
+    }
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        dropboxApiClient = getDropboxApiClient(context);
+    }
+
+    protected DbxClientV2 getDropboxApiClient(ProcessContext context) {
+        final DropboxCredentialService credentialService = 
context.getProperty(CREDENTIAL_SERVICE)
+                .asControllerService(DropboxCredentialService.class);
+        DbxRequestConfig config = new DbxRequestConfig(format("%s-%s", 
getClass().getSimpleName(), getIdentifier()));
+        DropboxCredentialDetails credential = 
credentialService.getDropboxCredential();
+        return new DbxClientV2(config, new 
DbxCredential(credential.getAccessToken(), -1L,
+                credential.getRefreshToken(), credential.getAppKey(), 
credential.getAppSecret()));
+    }
+
+    @Override
+    protected List<DropboxFileInfo> performListing(ProcessContext context, 
Long minTimestamp,
+            ListingMode listingMode) throws IOException {
+        final List<DropboxFileInfo> listing = new ArrayList<>();
+
+        final String folderName = 
context.getProperty(FOLDER_NAME).evaluateAttributeExpressions().getValue();
+        final Boolean recursive = 
context.getProperty(RECURSIVE_SEARCH).asBoolean();
+        final Long minAge = 
context.getProperty(MIN_AGE).asTimePeriod(TimeUnit.MILLISECONDS);
+
+        try {
+            ListFolderBuilder listFolderBuilder = 
dropboxApiClient.files().listFolderBuilder(folderName);
+            ListFolderResult result = listFolderBuilder
+                    .withRecursive(recursive)
+                    .start();
+
+            Predicate<Metadata> metadataFilter = metadata -> true;
+
+            if (minTimestamp != null && minTimestamp > 0) {
+                metadataFilter = metadataFilter.and(metadata ->
+                        ((FileMetadata) 
metadata).getServerModified().getTime()>= minTimestamp);
+            }
+
+            if (minAge != null && minAge > 0) {
+                long maxTimestamp = System.currentTimeMillis() - minAge;
+                metadataFilter = metadataFilter.and(metadata ->
+                        ((FileMetadata) 
metadata).getServerModified().getTime() < maxTimestamp);
+            }

Review Comment:
   I would suggest extracting this code into a separate 
`createMetadataFilter()` method. Also, I would call it at the very beginning of 
the `try` block in order to avoid overlapping with the Dropbox service call 
commands?



##########
nifi-nar-bundles/nifi-dropbox-bundle/nifi-dropbox-processors/src/test/java/org/apache/nifi/processors/dropbox/ListDropboxTest.java:
##########
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.nifi.processors.dropbox;
+
+import static org.apache.nifi.util.EqualsWrapper.wrapList;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.Mockito.when;
+
+import com.dropbox.core.v2.DbxClientV2;
+import com.dropbox.core.v2.files.DbxUserFilesRequests;
+import com.dropbox.core.v2.files.FileMetadata;
+import com.dropbox.core.v2.files.ListFolderBuilder;
+import com.dropbox.core.v2.files.ListFolderResult;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.function.Function;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.util.EqualsWrapper;
+import org.apache.nifi.util.MockPropertyValue;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+@ExtendWith(MockitoExtension.class)
+public class ListDropboxTest {
+
+    private ListDropbox testSubject;
+
+    @Mock
+    private ProcessContext mockProcessContext;
+    @Mock
+    private DbxClientV2 mockDbxClient;
+    @Mock
+    private DbxUserFilesRequests mockDbxUserFilesRequest;
+    @Mock
+    private ListFolderResult mockListFolderResult;
+    @Mock
+    private ListFolderBuilder mockListFolderBuilder;
+
+
+    @BeforeEach
+    void setUp() {
+        testSubject = new ListDropbox() {
+
+            @Override
+            public DbxClientV2 getDropboxApiClient(ProcessContext context) {
+                return mockDbxClient;
+            }
+        };
+        testSubject.onScheduled(mockProcessContext);
+    }
+
+    @Test
+    void testCreatedListableEntityContainsCorrectDataOldItemFiltered() throws 
Exception {
+        long minTimestamp = 1659707000;
+
+        String id1 = "id:11111";
+        String id2 = "id:22222";
+        String filename1 = "file_name_1";
+        String old_file_name = "old_file_name";
+        long size = 125;
+        long oldCreatedTime = 1657375066;
+        long createdTime = 1659707000;
+        String revision = "5e4ddb1320676a5c29261";
+
+        boolean isRecursive = true;
+        String folderName = "test_folder";
+
+        
when(mockProcessContext.getProperty(ListDropbox.FOLDER_NAME)).thenReturn(new 
MockPropertyValue(folderName));
+        when(mockProcessContext.getProperty(ListDropbox.RECURSIVE_SEARCH))
+                .thenReturn(new 
MockPropertyValue(String.valueOf(isRecursive)));
+        
when(mockProcessContext.getProperty(ListDropbox.MIN_AGE)).thenReturn(new 
MockPropertyValue("0 sec"));
+
+        when(mockDbxClient.files()).thenReturn(mockDbxUserFilesRequest);
+        
when(mockDbxUserFilesRequest.listFolderBuilder(folderName)).thenReturn(mockListFolderBuilder);
+        
when(mockListFolderBuilder.withRecursive(isRecursive)).thenReturn(mockListFolderBuilder);
+        when(mockListFolderBuilder.start()).thenReturn(mockListFolderResult);
+
+        when(mockListFolderResult.getEntries()).thenReturn(Arrays.asList(
+                createFileMetaData(filename1, id1, createdTime, revision, 
size),
+                createFileMetaData(old_file_name, id2, oldCreatedTime, 
revision, size)
+        ));
+        when(mockListFolderResult.getHasMore()).thenReturn(false);
+
+        List<DropboxFileInfo> expected = Collections.singletonList(
+                new DropboxFileInfo.Builder()
+                        .id(id1)
+                        .name(filename1)
+                        .size(size)
+                        .timestamp(createdTime)
+                        .revision(revision)
+                        .build()

Review Comment:
   `path` field should also be used in the test.



-- 
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-unsubscr...@nifi.apache.org

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

Reply via email to