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/30 19:20:33 UTC

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

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