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 2020/07/29 07:38:26 UTC

[GitHub] [nifi] turcsanyip opened a new pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

turcsanyip opened a new pull request #4438:
URL: https://github.com/apache/nifi/pull/4438


   Also added validator for Directory Name property in AbstractAzureDataLakeStorageProcessor
   
   https://issues.apache.org/jira/browse/NIFI-7340
   
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   In order to streamline the review of the contribution we ask you
   to ensure the following steps have been taken:
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

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


   @MuazmaZ Please try `mvn clean install -DskipTests ` in the project root.
   The BlobInfo error is due to changes in the parent class (`ListableEntity` in `nifi-processor-utils` module).
   I have no idea about the `git-commit-id-plugin` error. It is a recent change too and a full build might help too. Please first check if `.git` directory exists in your project root (it should if not deleted manually).


----------------------------------------------------------------
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.

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



[GitHub] [nifi] pvillard31 commented on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#issuecomment-668145633


   Merged to main, thanks @turcsanyip 


----------------------------------------------------------------
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.

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



[GitHub] [nifi] pgyori commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
pgyori commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463151518



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")

Review comment:
       I think it is more user-friendly to use RECURSE_SUBDIRECTORIES.getDisplayName() in the description.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] asfgit closed pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4438:
URL: https://github.com/apache/nifi/pull/4438


   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463879797



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();

Review comment:
       done

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();

Review comment:
       done




----------------------------------------------------------------
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.

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



[GitHub] [nifi] tpalfy commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463217516



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITListAzureDataLakeStorage.java
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertNotNull;
+
+public class ITListAzureDataLakeStorage extends AbstractAzureDataLakeStorageIT {
+
+    private Map<String, TestFile> testFiles;
+
+    @Override
+    protected Class<? extends Processor> getProcessorClass() {
+        return ListAzureDataLakeStorage.class;
+    }
+
+    @Before
+    public void setUp() {
+        testFiles = new HashMap<>();
+
+        TestFile testFile1 = new TestFile("", "file1");
+        uploadFile(testFile1);
+        testFiles.put(testFile1.getFilePath(), testFile1);
+
+        TestFile testFile2 = new TestFile("", "file2");
+        uploadFile(testFile2);
+        testFiles.put(testFile2.getFilePath(), testFile2);
+
+        TestFile testFile11 = new TestFile("dir1", "file11");
+        createDirectoryAndUploadFile(testFile11);
+        testFiles.put(testFile11.getFilePath(), testFile11);
+
+        TestFile testFile12 = new TestFile("dir1", "file12");
+        uploadFile(testFile12);
+        testFiles.put(testFile12.getFilePath(), testFile12);
+
+        TestFile testFile111 = new TestFile("dir1/dir11", "file111");
+        createDirectoryAndUploadFile(testFile111);
+        testFiles.put(testFile111.getFilePath(), testFile111);
+
+        TestFile testFile21 = new TestFile("dir 2", "file 21");
+        createDirectoryAndUploadFile(testFile21);
+        testFiles.put(testFile21.getFilePath(), testFile21);
+
+        createDirectory("dir3");
+    }
+
+    @Test
+    public void testListRootRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+
+        runProcessor();
+
+        assertSuccess("file1", "file2", "dir1/file11", "dir1/file12", "dir1/dir11/file111", "dir 2/file 21");
+    }
+
+    @Test
+    public void testListRootNonRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.RECURSE_SUBDIRECTORIES, "false");
+
+        runProcessor();
+
+        assertSuccess("file1", "file2");
+    }
+
+    @Test
+    public void testListSubdirectoryRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/file12", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListSubdirectoryNonRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+        runner.setProperty(ListAzureDataLakeStorage.RECURSE_SUBDIRECTORIES, "false");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/file12");
+    }
+
+    @Test
+    public void testListWithFileFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.FILE_FILTER, "file1.*");
+
+        runProcessor();
+
+        assertSuccess("file1", "dir1/file11", "dir1/file12", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListRootWithPathFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.PATH_FILTER, "dir1.*");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/file12", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListSubdirectoryWithPathFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+        runner.setProperty(ListAzureDataLakeStorage.PATH_FILTER, "dir1.*");
+
+        runProcessor();
+
+        assertSuccess("dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListRootWithFileAndPathFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.FILE_FILTER, ".*11");
+        runner.setProperty(ListAzureDataLakeStorage.PATH_FILTER, "dir1.*");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListEmptyDirectory() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir3");
+
+        runProcessor();
+
+        assertSuccess();
+    }
+
+    @Test
+    public void testListNonExistingDirectory() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dummy");
+
+        runProcessor();
+
+        assertFailure();
+    }
+
+    @Test
+    public void testListWithNonExistingFileSystem() {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.FILESYSTEM, "dummy");
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+
+        runProcessor();
+
+        assertFailure();
+    }
+
+    @Test
+    public void testListWithRecords() throws InitializationException {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+
+        MockRecordWriter recordWriter = new MockRecordWriter(null, false);
+        runner.addControllerService("record-writer", recordWriter);
+        runner.enableControllerService(recordWriter);
+        runner.setProperty(ListAzureDataLakeStorage.RECORD_WRITER, "record-writer");
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ListAzureDataLakeStorage.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ListAzureDataLakeStorage.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals("record.count", "3");
+    }
+
+    private void runProcessor() {
+        runner.assertValid();
+        runner.run();
+    }
+
+    private void assertSuccess(String... testFilePaths) throws Exception {
+        runner.assertTransferCount(ListAzureDataLakeStorage.REL_SUCCESS, testFilePaths.length);
+
+        Map<String, TestFile> expectedFiles = new HashMap<>(testFiles);
+        expectedFiles.keySet().retainAll(Arrays.asList(testFilePaths));
+
+        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ListAzureDataLakeStorage.REL_SUCCESS);
+
+        for (MockFlowFile flowFile : flowFiles) {
+            String filePath = flowFile.getAttribute("azure.filePath");
+            TestFile testFile = expectedFiles.remove(filePath);
+            assertNotNull("File path not found in the expected map", testFile);
+            assertFlowFile(testFile, flowFile);
+        }
+    }
+
+    private void assertFlowFile(TestFile testFile, MockFlowFile flowFile) throws Exception {
+        flowFile.assertAttributeEquals("azure.filesystem", fileSystemName);
+        flowFile.assertAttributeEquals("azure.filePath", testFile.getFilePath());
+        flowFile.assertAttributeEquals("azure.directory", testFile.getDirectory());
+        flowFile.assertAttributeEquals("azure.filename", testFile.getFilename());
+        flowFile.assertAttributeEquals("azure.length", String.valueOf(testFile.getFileContent().length()));
+
+        flowFile.assertAttributeExists("azure.lastModified");
+        flowFile.assertAttributeExists("azure.etag");
+
+        flowFile.assertContentEquals("");
+    }
+
+    private void assertFailure() {

Review comment:
       `assertFailure` is misleading imo (as there is no `failure` relationship).
   Could call `assertNoop` or `assertIgnore` instead.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")

Review comment:
       Minor: `"will be listed"` would feel more natural.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();
+        pathPattern = pathFilter != null ? Pattern.compile(pathFilter) : null;
+    }
+
+    @OnStopped
+    public void onStopped() {
+        filePattern = null;
+        pathPattern = null;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext context, Collection<ValidationResult> results) {
+        if (context.getProperty(PATH_FILTER).isSet() && !context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean()) {
+            results.add(new ValidationResult.Builder()
+                    .subject(PATH_FILTER.getDisplayName())
+                    .valid(false)
+                    .explanation(String.format("'%s' cannot be set when '%s' is false", PATH_FILTER.getDisplayName(), RECURSE_SUBDIRECTORIES.getDisplayName()))
+                    .build());
+        }
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return ADLSFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected Scope getStateScope(PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(PropertyDescriptor property) {
+        return ADLS_CREDENTIALS_SERVICE.equals(property)

Review comment:
       Shouldn't we consider more properties here?
   `RECURSE_SUBDIRECTORIES`, `FILE_FILTER`, `PATH_FILTER` and `LISTING_STRATEGY` come to my mind.
   Also could use a `Collection.contains()`.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureDataLakeStorageIT.java
##########
@@ -82,9 +88,51 @@ protected void uploadFile(String directory, String filename, String fileContent)
         fileClient.flush(fileContentBytes.length);
     }
 
+    protected void uploadFile(TestFile testFile) {
+        uploadFile(testFile.getDirectory(), testFile.getFilename(), testFile.getFileContent());
+    }
+
     protected void createDirectoryAndUploadFile(String directory, String filename, String fileContent) {
-        fileSystemClient.createDirectory(directory);
+        createDirectory(directory);
 
         uploadFile(directory, filename, fileContent);
     }
+
+    protected void createDirectoryAndUploadFile(TestFile testFile) {
+        createDirectoryAndUploadFile(testFile.getDirectory(), testFile.getFilename(), testFile.getFileContent());
+    }
+
+    protected static class TestFile {
+        private final String directory;
+        private final String filename;
+        private final String fileContent;
+
+        public TestFile(String directory, String filename, String fileContent) {
+            this.directory = directory;
+            this.filename = filename;
+            this.fileContent = fileContent;
+        }
+
+        public TestFile(String directory, String filename) {
+            this.directory = directory;
+            this.filename = filename;
+            this.fileContent = TEST_FILE_CONTENT;
+        }

Review comment:
       ```suggestion
           public TestFile(String directory, String filename) {
               this(directory, filename, TEST_FILE_CONTENT);
           }
   
           public TestFile(String directory, String filename, String fileContent) {
               this.directory = directory;
               this.filename = filename;
               this.fileContent = fileContent;
           }
   ```




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463880263



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureDataLakeStorageProcessor.java
##########
@@ -146,8 +154,22 @@ public static DataLakeServiceClient getStorageClient(PropertyContext context, Fl
         return storageClient;
     }
 
-    @Override
-    public Set<Relationship> getRelationships() {
-        return RELATIONSHIPS;
+    private static class DirectoryValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            ValidationResult.Builder builder = new ValidationResult.Builder()
+                    .subject(DIRECTORY.getDisplayName())
+                    .input(input);
+
+            if (context.isExpressionLanguagePresent(input)) {
+                builder.valid(true).explanation("Expression Language Present");

Review comment:
       Added runtime check.
   Also moved the runtime check methods to the abstract parent class.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463880021



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();
+        pathPattern = pathFilter != null ? Pattern.compile(pathFilter) : null;
+    }
+
+    @OnStopped
+    public void onStopped() {
+        filePattern = null;
+        pathPattern = null;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext context, Collection<ValidationResult> results) {
+        if (context.getProperty(PATH_FILTER).isSet() && !context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean()) {
+            results.add(new ValidationResult.Builder()
+                    .subject(PATH_FILTER.getDisplayName())
+                    .valid(false)
+                    .explanation(String.format("'%s' cannot be set when '%s' is false", PATH_FILTER.getDisplayName(), RECURSE_SUBDIRECTORIES.getDisplayName()))
+                    .build());
+        }
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return ADLSFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected Scope getStateScope(PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(PropertyDescriptor property) {
+        return ADLS_CREDENTIALS_SERVICE.equals(property)
+                || FILESYSTEM.equals(property)
+                || DIRECTORY.equals(property);
+    }
+
+    @Override
+    protected String getPath(ProcessContext context) {
+        String directory = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
+        return directory != null ? directory : ".";
+    }
+
+    @Override
+    protected List<ADLSFileInfo> performListing(ProcessContext context, Long minTimestamp) throws IOException {
+        try {
+            String fileSystem = context.getProperty(FILESYSTEM).evaluateAttributeExpressions().getValue();
+            String baseDirectory = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
+            boolean recurseSubdirectories = context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean();
+
+            if (StringUtils.isBlank(fileSystem)) {
+                throw new ProcessException(FILESYSTEM.getDisplayName() + " property evaluated to empty string. " +
+                        FILESYSTEM.getDisplayName() + " must be specified as a non-empty string.");
+            }

Review comment:
       Changed the validator.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] jfrazee commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
jfrazee commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463293125



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")

Review comment:
       These get re-used in `createAttributes()` so maybe worth making them constants.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463880311



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")

Review comment:
       done




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r464508508



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();
+        pathPattern = pathFilter != null ? Pattern.compile(pathFilter) : null;
+    }
+
+    @OnStopped
+    public void onStopped() {
+        filePattern = null;
+        pathPattern = null;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext context, Collection<ValidationResult> results) {
+        if (context.getProperty(PATH_FILTER).isSet() && !context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean()) {
+            results.add(new ValidationResult.Builder()
+                    .subject(PATH_FILTER.getDisplayName())
+                    .valid(false)
+                    .explanation(String.format("'%s' cannot be set when '%s' is false", PATH_FILTER.getDisplayName(), RECURSE_SUBDIRECTORIES.getDisplayName()))
+                    .build());
+        }
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return ADLSFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected Scope getStateScope(PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(PropertyDescriptor property) {
+        return ADLS_CREDENTIALS_SERVICE.equals(property)

Review comment:
       Added properties, using set.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] MuazmaZ commented on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
MuazmaZ commented on pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#issuecomment-665186559






----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463879731



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()

Review comment:
       done




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463880452



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")

Review comment:
       Created constants and using them in all ADLS processors.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

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


   @MuazmaZ Thanks for the review and testing.
   I created follow-up jira for the mime.type: https://issues.apache.org/jira/browse/NIFI-7701


----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

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


   Found a bug in the "Tracking Entities" listing strategy that has been fixed.
   All other review comments addressed. Please check them.


----------------------------------------------------------------
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.

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



[GitHub] [nifi] pvillard31 commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r461579592



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()

Review comment:
       Just in case I'd add support for variable registry scope for expression language.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()

Review comment:
       Just in case I'd add support for variable registry scope for expression language.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();

Review comment:
       Evaluate EL for variable registry.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();

Review comment:
       Evaluate EL for variable registry.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()

Review comment:
       yeah I guess the issue might be about not mixing the regex and the EL evaluations with the ``$`` sign? I think we have something similar in ReplaceText processor or a similar processor.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r464509459



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/ITListAzureDataLakeStorage.java
##########
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.serialization.record.MockRecordWriter;
+import org.apache.nifi.util.MockFlowFile;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertNotNull;
+
+public class ITListAzureDataLakeStorage extends AbstractAzureDataLakeStorageIT {
+
+    private Map<String, TestFile> testFiles;
+
+    @Override
+    protected Class<? extends Processor> getProcessorClass() {
+        return ListAzureDataLakeStorage.class;
+    }
+
+    @Before
+    public void setUp() {
+        testFiles = new HashMap<>();
+
+        TestFile testFile1 = new TestFile("", "file1");
+        uploadFile(testFile1);
+        testFiles.put(testFile1.getFilePath(), testFile1);
+
+        TestFile testFile2 = new TestFile("", "file2");
+        uploadFile(testFile2);
+        testFiles.put(testFile2.getFilePath(), testFile2);
+
+        TestFile testFile11 = new TestFile("dir1", "file11");
+        createDirectoryAndUploadFile(testFile11);
+        testFiles.put(testFile11.getFilePath(), testFile11);
+
+        TestFile testFile12 = new TestFile("dir1", "file12");
+        uploadFile(testFile12);
+        testFiles.put(testFile12.getFilePath(), testFile12);
+
+        TestFile testFile111 = new TestFile("dir1/dir11", "file111");
+        createDirectoryAndUploadFile(testFile111);
+        testFiles.put(testFile111.getFilePath(), testFile111);
+
+        TestFile testFile21 = new TestFile("dir 2", "file 21");
+        createDirectoryAndUploadFile(testFile21);
+        testFiles.put(testFile21.getFilePath(), testFile21);
+
+        createDirectory("dir3");
+    }
+
+    @Test
+    public void testListRootRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+
+        runProcessor();
+
+        assertSuccess("file1", "file2", "dir1/file11", "dir1/file12", "dir1/dir11/file111", "dir 2/file 21");
+    }
+
+    @Test
+    public void testListRootNonRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.RECURSE_SUBDIRECTORIES, "false");
+
+        runProcessor();
+
+        assertSuccess("file1", "file2");
+    }
+
+    @Test
+    public void testListSubdirectoryRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/file12", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListSubdirectoryNonRecursive() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+        runner.setProperty(ListAzureDataLakeStorage.RECURSE_SUBDIRECTORIES, "false");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/file12");
+    }
+
+    @Test
+    public void testListWithFileFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.FILE_FILTER, "file1.*");
+
+        runProcessor();
+
+        assertSuccess("file1", "dir1/file11", "dir1/file12", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListRootWithPathFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.PATH_FILTER, "dir1.*");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/file12", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListSubdirectoryWithPathFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+        runner.setProperty(ListAzureDataLakeStorage.PATH_FILTER, "dir1.*");
+
+        runProcessor();
+
+        assertSuccess("dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListRootWithFileAndPathFilter() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+        runner.setProperty(ListAzureDataLakeStorage.FILE_FILTER, ".*11");
+        runner.setProperty(ListAzureDataLakeStorage.PATH_FILTER, "dir1.*");
+
+        runProcessor();
+
+        assertSuccess("dir1/file11", "dir1/dir11/file111");
+    }
+
+    @Test
+    public void testListEmptyDirectory() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir3");
+
+        runProcessor();
+
+        assertSuccess();
+    }
+
+    @Test
+    public void testListNonExistingDirectory() throws Exception {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dummy");
+
+        runProcessor();
+
+        assertFailure();
+    }
+
+    @Test
+    public void testListWithNonExistingFileSystem() {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.FILESYSTEM, "dummy");
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "");
+
+        runProcessor();
+
+        assertFailure();
+    }
+
+    @Test
+    public void testListWithRecords() throws InitializationException {
+        runner.setProperty(AbstractAzureDataLakeStorageProcessor.DIRECTORY, "dir1");
+
+        MockRecordWriter recordWriter = new MockRecordWriter(null, false);
+        runner.addControllerService("record-writer", recordWriter);
+        runner.enableControllerService(recordWriter);
+        runner.setProperty(ListAzureDataLakeStorage.RECORD_WRITER, "record-writer");
+
+        runner.run();
+
+        runner.assertAllFlowFilesTransferred(ListAzureDataLakeStorage.REL_SUCCESS, 1);
+        MockFlowFile flowFile = runner.getFlowFilesForRelationship(ListAzureDataLakeStorage.REL_SUCCESS).get(0);
+        flowFile.assertAttributeEquals("record.count", "3");
+    }
+
+    private void runProcessor() {
+        runner.assertValid();
+        runner.run();
+    }
+
+    private void assertSuccess(String... testFilePaths) throws Exception {
+        runner.assertTransferCount(ListAzureDataLakeStorage.REL_SUCCESS, testFilePaths.length);
+
+        Map<String, TestFile> expectedFiles = new HashMap<>(testFiles);
+        expectedFiles.keySet().retainAll(Arrays.asList(testFilePaths));
+
+        List<MockFlowFile> flowFiles = runner.getFlowFilesForRelationship(ListAzureDataLakeStorage.REL_SUCCESS);
+
+        for (MockFlowFile flowFile : flowFiles) {
+            String filePath = flowFile.getAttribute("azure.filePath");
+            TestFile testFile = expectedFiles.remove(filePath);
+            assertNotNull("File path not found in the expected map", testFile);
+            assertFlowFile(testFile, flowFile);
+        }
+    }
+
+    private void assertFlowFile(TestFile testFile, MockFlowFile flowFile) throws Exception {
+        flowFile.assertAttributeEquals("azure.filesystem", fileSystemName);
+        flowFile.assertAttributeEquals("azure.filePath", testFile.getFilePath());
+        flowFile.assertAttributeEquals("azure.directory", testFile.getDirectory());
+        flowFile.assertAttributeEquals("azure.filename", testFile.getFilename());
+        flowFile.assertAttributeEquals("azure.length", String.valueOf(testFile.getFileContent().length()));
+
+        flowFile.assertAttributeExists("azure.lastModified");
+        flowFile.assertAttributeExists("azure.etag");
+
+        flowFile.assertContentEquals("");
+    }
+
+    private void assertFailure() {

Review comment:
       It is a failure case without sending FF to failure (List processors typically have only success and do not send FF on error).
   Added assert for checking the error logging.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r461881207



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()

Review comment:
       I tried it but `REGULAR_EXPRESSION_VALIDATOR` does not support EL for some reason:
   https://github.com/apache/nifi/blob/cee6cfddf2e1918a1edda5fb1e02c151663a2d97/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/processor/util/StandardValidators.java#L415
   
   It seems it is quite easy to define a new `REGULAR_EXPRESSION_WITH_EL_VALIDATOR` (just passing true in the 3rd argument to `createRegexValidator()`) so I'll give it a try again.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();
+        pathPattern = pathFilter != null ? Pattern.compile(pathFilter) : null;
+    }
+
+    @OnStopped
+    public void onStopped() {
+        filePattern = null;
+        pathPattern = null;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext context, Collection<ValidationResult> results) {
+        if (context.getProperty(PATH_FILTER).isSet() && !context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean()) {
+            results.add(new ValidationResult.Builder()
+                    .subject(PATH_FILTER.getDisplayName())
+                    .valid(false)
+                    .explanation(String.format("'%s' cannot be set when '%s' is false", PATH_FILTER.getDisplayName(), RECURSE_SUBDIRECTORIES.getDisplayName()))
+                    .build());
+        }
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return ADLSFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected Scope getStateScope(PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(PropertyDescriptor property) {
+        return ADLS_CREDENTIALS_SERVICE.equals(property)
+                || FILESYSTEM.equals(property)
+                || DIRECTORY.equals(property);
+    }
+
+    @Override
+    protected String getPath(ProcessContext context) {
+        String directory = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
+        return directory != null ? directory : ".";
+    }
+
+    @Override
+    protected List<ADLSFileInfo> performListing(ProcessContext context, Long minTimestamp) throws IOException {
+        try {
+            String fileSystem = context.getProperty(FILESYSTEM).evaluateAttributeExpressions().getValue();
+            String baseDirectory = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
+            boolean recurseSubdirectories = context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean();
+
+            if (StringUtils.isBlank(fileSystem)) {
+                throw new ProcessException(FILESYSTEM.getDisplayName() + " property evaluated to empty string. " +
+                        FILESYSTEM.getDisplayName() + " must be specified as a non-empty string.");
+            }

Review comment:
       Indeed, it should. Will change it. Thanks

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureDataLakeStorageProcessor.java
##########
@@ -146,8 +154,22 @@ public static DataLakeServiceClient getStorageClient(PropertyContext context, Fl
         return storageClient;
     }
 
-    @Override
-    public Set<Relationship> getRelationships() {
-        return RELATIONSHIPS;
+    private static class DirectoryValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            ValidationResult.Builder builder = new ValidationResult.Builder()
+                    .subject(DIRECTORY.getDisplayName())
+                    .input(input);
+
+            if (context.isExpressionLanguagePresent(input)) {
+                builder.valid(true).explanation("Expression Language Present");
+            } else if (input.startsWith("/")) {
+                builder.valid(false).explanation(String.format("'%s' cannot contain a leading '/'", DIRECTORY.getDisplayName()));
+            } else {
+                builder.valid(true);

Review comment:
       Good point, we can catch the whitespace-only value here. Will add it.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] jfrazee commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
jfrazee commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r461795133



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = Collections.unmodifiableList(Arrays.asList(
+            ADLS_CREDENTIALS_SERVICE,
+            FILESYSTEM,
+            DIRECTORY,
+            RECURSE_SUBDIRECTORIES,
+            FILE_FILTER,
+            PATH_FILTER,
+            RECORD_WRITER,
+            LISTING_STRATEGY,
+            TRACKING_STATE_CACHE,
+            TRACKING_TIME_WINDOW,
+            INITIAL_LISTING_TARGET));
+
+    private volatile Pattern filePattern;
+    private volatile Pattern pathPattern;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return PROPERTIES;
+    }
+
+    @OnScheduled
+    public void onScheduled(ProcessContext context) {
+        String fileFilter = context.getProperty(FILE_FILTER).getValue();
+        filePattern = fileFilter != null ? Pattern.compile(fileFilter) : null;
+
+        String pathFilter = context.getProperty(PATH_FILTER).getValue();
+        pathPattern = pathFilter != null ? Pattern.compile(pathFilter) : null;
+    }
+
+    @OnStopped
+    public void onStopped() {
+        filePattern = null;
+        pathPattern = null;
+    }
+
+    @Override
+    protected void customValidate(ValidationContext context, Collection<ValidationResult> results) {
+        if (context.getProperty(PATH_FILTER).isSet() && !context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean()) {
+            results.add(new ValidationResult.Builder()
+                    .subject(PATH_FILTER.getDisplayName())
+                    .valid(false)
+                    .explanation(String.format("'%s' cannot be set when '%s' is false", PATH_FILTER.getDisplayName(), RECURSE_SUBDIRECTORIES.getDisplayName()))
+                    .build());
+        }
+    }
+
+    @Override
+    protected RecordSchema getRecordSchema() {
+        return ADLSFileInfo.getRecordSchema();
+    }
+
+    @Override
+    protected Scope getStateScope(PropertyContext context) {
+        return Scope.CLUSTER;
+    }
+
+    @Override
+    protected String getDefaultTimePrecision() {
+        return PRECISION_SECONDS.getValue();
+    }
+
+    @Override
+    protected boolean isListingResetNecessary(PropertyDescriptor property) {
+        return ADLS_CREDENTIALS_SERVICE.equals(property)
+                || FILESYSTEM.equals(property)
+                || DIRECTORY.equals(property);
+    }
+
+    @Override
+    protected String getPath(ProcessContext context) {
+        String directory = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
+        return directory != null ? directory : ".";
+    }
+
+    @Override
+    protected List<ADLSFileInfo> performListing(ProcessContext context, Long minTimestamp) throws IOException {
+        try {
+            String fileSystem = context.getProperty(FILESYSTEM).evaluateAttributeExpressions().getValue();
+            String baseDirectory = context.getProperty(DIRECTORY).evaluateAttributeExpressions().getValue();
+            boolean recurseSubdirectories = context.getProperty(RECURSE_SUBDIRECTORIES).asBoolean();
+
+            if (StringUtils.isBlank(fileSystem)) {
+                throw new ProcessException(FILESYSTEM.getDisplayName() + " property evaluated to empty string. " +
+                        FILESYSTEM.getDisplayName() + " must be specified as a non-empty string.");
+            }

Review comment:
       This makes me wonder, should the `FILESYSTEM` property be using `NON_BLANK_VALIDATOR` instead of `NON_EMPTY_VALIDATOR`?

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureDataLakeStorageProcessor.java
##########
@@ -146,8 +154,22 @@ public static DataLakeServiceClient getStorageClient(PropertyContext context, Fl
         return storageClient;
     }
 
-    @Override
-    public Set<Relationship> getRelationships() {
-        return RELATIONSHIPS;
+    private static class DirectoryValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            ValidationResult.Builder builder = new ValidationResult.Builder()
+                    .subject(DIRECTORY.getDisplayName())
+                    .input(input);
+
+            if (context.isExpressionLanguagePresent(input)) {
+                builder.valid(true).explanation("Expression Language Present");
+            } else if (input.startsWith("/")) {
+                builder.valid(false).explanation(String.format("'%s' cannot contain a leading '/'", DIRECTORY.getDisplayName()));
+            } else {
+                builder.valid(true);

Review comment:
       I think this should be `builder.valid(StringUtils.nonBlank(input))` or similar to catch it early. If you run this with non-zero length whitespace you'll get an error at run time.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] MuazmaZ edited a comment on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
MuazmaZ edited a comment on pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#issuecomment-665449257


   @turcsanyip @pvillard31 I have not been able to build this PR getting this ERROR: 
   [ERROR] Failed to execute goal pl.project13.maven:git-commit-id-plugin:4.0.0:revision (default) on project nifi-api: .git directory is not found! Please specify a valid [dotGitDirectory] in your pom.xml
   For Azure bundle I am getting: ..\processors\src\main\java\org\apache\nifi\processors\azure\storage\utils\BlobInfo.java:[121,4] error: method does not override or implement a method from a supertype
   


----------------------------------------------------------------
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.

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



[GitHub] [nifi] pgyori commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
pgyori commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463154726



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureDataLakeStorageProcessor.java
##########
@@ -146,8 +154,22 @@ public static DataLakeServiceClient getStorageClient(PropertyContext context, Fl
         return storageClient;
     }
 
-    @Override
-    public Set<Relationship> getRelationships() {
-        return RELATIONSHIPS;
+    private static class DirectoryValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            ValidationResult.Builder builder = new ValidationResult.Builder()
+                    .subject(DIRECTORY.getDisplayName())
+                    .input(input);
+
+            if (context.isExpressionLanguagePresent(input)) {
+                builder.valid(true).explanation("Expression Language Present");

Review comment:
       A runtime check (in onTrigger) for the actual value (after evaluating the expression language) might be necessary.




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463879602



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()

Review comment:
       Added `REGULAR_EXPRESSION_WITH_EL_VALIDATOR`




----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463880093



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/AbstractAzureDataLakeStorageProcessor.java
##########
@@ -146,8 +154,22 @@ public static DataLakeServiceClient getStorageClient(PropertyContext context, Fl
         return storageClient;
     }
 
-    @Override
-    public Set<Relationship> getRelationships() {
-        return RELATIONSHIPS;
+    private static class DirectoryValidator implements Validator {
+        @Override
+        public ValidationResult validate(String subject, String input, ValidationContext context) {
+            ValidationResult.Builder builder = new ValidationResult.Builder()
+                    .subject(DIRECTORY.getDisplayName())
+                    .input(input);
+
+            if (context.isExpressionLanguagePresent(input)) {
+                builder.valid(true).explanation("Expression Language Present");
+            } else if (input.startsWith("/")) {
+                builder.valid(false).explanation(String.format("'%s' cannot contain a leading '/'", DIRECTORY.getDisplayName()));
+            } else {
+                builder.valid(true);

Review comment:
       Added whitespace-only check.

##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/main/java/org/apache/nifi/processors/azure/storage/ListAzureDataLakeStorage.java
##########
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.azure.storage;
+
+import com.azure.storage.file.datalake.DataLakeFileSystemClient;
+import com.azure.storage.file.datalake.DataLakeServiceClient;
+import com.azure.storage.file.datalake.models.ListPathsOptions;
+import org.apache.commons.lang3.RegExUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.exception.ExceptionUtils;
+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.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.annotation.lifecycle.OnStopped;
+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.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processors.azure.storage.utils.ADLSFileInfo;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.INITIAL_LISTING_TARGET;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_STATE_CACHE;
+import static org.apache.nifi.processor.util.list.ListedEntityTracker.TRACKING_TIME_WINDOW;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.ADLS_CREDENTIALS_SERVICE;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.DIRECTORY;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.FILESYSTEM;
+import static org.apache.nifi.processors.azure.AbstractAzureDataLakeStorageProcessor.getStorageClient;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"azure", "microsoft", "cloud", "storage", "adlsgen2", "datalake"})
+@SeeAlso({PutAzureDataLakeStorage.class, DeleteAzureDataLakeStorage.class, FetchAzureDataLakeStorage.class})
+@CapabilityDescription("Lists directory in an Azure Data Lake Storage Gen 2 filesystem")
+@WritesAttributes({
+        @WritesAttribute(attribute = "azure.filesystem", description = "The name of the Azure File System"),
+        @WritesAttribute(attribute = "azure.filePath", description = "The full path of the Azure File"),
+        @WritesAttribute(attribute = "azure.directory", description = "The name of the Azure Directory"),
+        @WritesAttribute(attribute = "azure.filename", description = "The name of the Azure File"),
+        @WritesAttribute(attribute = "azure.length", description = "The length of the Azure File"),
+        @WritesAttribute(attribute = "azure.lastModified", description = "The last modification time of the Azure File"),
+        @WritesAttribute(attribute = "azure.etag", description = "The ETag of the Azure File")
+})
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@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 ListAzureDataLakeStorage extends AbstractListProcessor<ADLSFileInfo> {
+
+    public static final PropertyDescriptor RECURSE_SUBDIRECTORIES = new PropertyDescriptor.Builder()
+            .name("recurse-subdirectories")
+            .displayName("Recurse Subdirectories")
+            .description("Indicates whether to list files from subdirectories of the directory")
+            .required(true)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .build();
+
+    public static final PropertyDescriptor FILE_FILTER = new PropertyDescriptor.Builder()
+            .name("file-filter")
+            .displayName("File Filter")
+            .description("Only files whose names match the given regular expression will be picked up")
+            .required(false)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PATH_FILTER = new PropertyDescriptor.Builder()
+            .name("path-filter")
+            .displayName("Path Filter")
+            .description("When " + RECURSE_SUBDIRECTORIES.getName() + " is true, then only subdirectories whose paths match the given regular expression will be scanned")

Review comment:
       done




----------------------------------------------------------------
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.

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



[GitHub] [nifi] pvillard31 commented on pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#issuecomment-665109263


   Code change LGTM, left a very minor comment, but I won't have the chance to give it a try before a while.
   @MuazmaZ - I know you looked at previous ADLS related work, are you able to have a look at it?


----------------------------------------------------------------
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.

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



[GitHub] [nifi] turcsanyip commented on a change in pull request #4438: NIFI-7340: Adding ListAzureDataLakeStorage

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4438:
URL: https://github.com/apache/nifi/pull/4438#discussion_r463880354



##########
File path: nifi-nar-bundles/nifi-azure-bundle/nifi-azure-processors/src/test/java/org/apache/nifi/processors/azure/storage/AbstractAzureDataLakeStorageIT.java
##########
@@ -82,9 +88,51 @@ protected void uploadFile(String directory, String filename, String fileContent)
         fileClient.flush(fileContentBytes.length);
     }
 
+    protected void uploadFile(TestFile testFile) {
+        uploadFile(testFile.getDirectory(), testFile.getFilename(), testFile.getFileContent());
+    }
+
     protected void createDirectoryAndUploadFile(String directory, String filename, String fileContent) {
-        fileSystemClient.createDirectory(directory);
+        createDirectory(directory);
 
         uploadFile(directory, filename, fileContent);
     }
+
+    protected void createDirectoryAndUploadFile(TestFile testFile) {
+        createDirectoryAndUploadFile(testFile.getDirectory(), testFile.getFilename(), testFile.getFileContent());
+    }
+
+    protected static class TestFile {
+        private final String directory;
+        private final String filename;
+        private final String fileContent;
+
+        public TestFile(String directory, String filename, String fileContent) {
+            this.directory = directory;
+            this.filename = filename;
+            this.fileContent = fileContent;
+        }
+
+        public TestFile(String directory, String filename) {
+            this.directory = directory;
+            this.filename = filename;
+            this.fileContent = TEST_FILE_CONTENT;
+        }

Review comment:
       done




----------------------------------------------------------------
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.

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