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

[GitHub] [nifi] turcsanyip commented on a diff in pull request #6192: NIFI-10212 added ListSmb processor and SmbConnectionPoolService

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


##########
nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/ListSmb.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.smb;
+
+import static java.time.format.DateTimeFormatter.ISO_DATE_TIME;
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableMap;
+import static org.apache.nifi.components.state.Scope.CLUSTER;
+import static org.apache.nifi.processor.util.StandardValidators.INTEGER_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR;
+
+import java.io.IOException;
+import java.net.URI;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.services.smb.SmbConnectionPoolService;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"microsoft", "storage", "samba"})
+@SeeAlso({PutSmbFile.class, GetSmbFile.class})
+@CapabilityDescription("Retrieves a listing of files shared via SMB protocol. For each file that is listed, " +
+        "creates a FlowFile that represents the file. This Processor is designed to run on Primary Node only in " +
+        "a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left " +
+        "off without duplicating all of the data.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({
+        @WritesAttribute(attribute = "filename", description = "The name of the file that was read from filesystem."),
+        @WritesAttribute(attribute = "path", description =
+                "The path is set to the relative path of the file's directory "
+                        + "on filesystem compared to the Share and Input Directory properties and the configured host "
+                        + "and port inherited from the configured connection pool controller service. For example, for "
+                        + "a given remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listed from "
+                        + "smb://HOSTNAME:PORT/SHARE:DIRECTORY\\sub\\folder\\file then the path attribute will be set to \"sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "absolute.path", description =
+                "The absolute.path is set to the absolute path of the file's directory on the remote location. For example, "
+                        + "given a remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listen from "
+                        + "SHARE:\\DIRECTORY\\sub\\folder\\file then the absolute.path attribute will be set to "
+                        + "\"SHARE:\\DIRECTORY\\sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "identifier", description =
+                "The identifier of the file. This equals to the path attribute so two files with the same relative path "
+                        + "coming from different file shares considered to be identical."),
+        @WritesAttribute(attribute = "timestamp", description =
+                "The timestamp of when the file in the filesystem was modified as 'yyyy-MM-dd'T'HH:mm:ssZ'"),
+        @WritesAttribute(attribute = "size", description = "The number of bytes in the source file"),
+})
+@Stateful(scopes = {Scope.CLUSTER}, description =
+        "After performing a listing of files, the state of the previous listing can be stored in order to list files "
+                + "continuously without duplication."
+)
+public class ListSmb extends AbstractListProcessor<SmbListableEntity> {
+
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+            .displayName("Input Directory")
+            .name("directory")
+            .description("The network folder to which files should be written. This is the remaining relative " +
+                    "after the hostname: smb://HOSTNAME:PORT/SHARE/[DIRECTORY]\\sub\\directories. It is also possible "
+                    + " to add subdirectories using this property. The given path on the remote file share must exists. "
+                    + "The existence of the remote folder can be checked using verification. You may mix different "
+                    + "directory separators in this property. If so NiFi will unify all of them and will use windows's"
+                    + "directory separator: '\\' ")
+            .required(false)
+            .addValidator(NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MINIMUM_AGE = new PropertyDescriptor.Builder()

Review Comment:
   I would consider to add Maximum File Age and Minimum/Maximum File Size proprties too (as in case of some other List processors).



##########
nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/ListSmb.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.smb;
+
+import static java.time.format.DateTimeFormatter.ISO_DATE_TIME;
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableMap;
+import static org.apache.nifi.components.state.Scope.CLUSTER;
+import static org.apache.nifi.processor.util.StandardValidators.INTEGER_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR;
+
+import java.io.IOException;
+import java.net.URI;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.services.smb.SmbConnectionPoolService;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"microsoft", "storage", "samba"})
+@SeeAlso({PutSmbFile.class, GetSmbFile.class})
+@CapabilityDescription("Retrieves a listing of files shared via SMB protocol. For each file that is listed, " +
+        "creates a FlowFile that represents the file. This Processor is designed to run on Primary Node only in " +
+        "a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left " +
+        "off without duplicating all of the data.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({
+        @WritesAttribute(attribute = "filename", description = "The name of the file that was read from filesystem."),
+        @WritesAttribute(attribute = "path", description =
+                "The path is set to the relative path of the file's directory "
+                        + "on filesystem compared to the Share and Input Directory properties and the configured host "
+                        + "and port inherited from the configured connection pool controller service. For example, for "
+                        + "a given remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listed from "
+                        + "smb://HOSTNAME:PORT/SHARE:DIRECTORY\\sub\\folder\\file then the path attribute will be set to \"sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "absolute.path", description =
+                "The absolute.path is set to the absolute path of the file's directory on the remote location. For example, "
+                        + "given a remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listen from "
+                        + "SHARE:\\DIRECTORY\\sub\\folder\\file then the absolute.path attribute will be set to "
+                        + "\"SHARE:\\DIRECTORY\\sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "identifier", description =
+                "The identifier of the file. This equals to the path attribute so two files with the same relative path "
+                        + "coming from different file shares considered to be identical."),
+        @WritesAttribute(attribute = "timestamp", description =
+                "The timestamp of when the file in the filesystem was modified as 'yyyy-MM-dd'T'HH:mm:ssZ'"),
+        @WritesAttribute(attribute = "size", description = "The number of bytes in the source file"),
+})
+@Stateful(scopes = {Scope.CLUSTER}, description =
+        "After performing a listing of files, the state of the previous listing can be stored in order to list files "
+                + "continuously without duplication."
+)
+public class ListSmb extends AbstractListProcessor<SmbListableEntity> {
+
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+            .displayName("Input Directory")
+            .name("directory")
+            .description("The network folder to which files should be written. This is the remaining relative " +
+                    "after the hostname: smb://HOSTNAME:PORT/SHARE/[DIRECTORY]\\sub\\directories. It is also possible "
+                    + " to add subdirectories using this property. The given path on the remote file share must exists. "
+                    + "The existence of the remote folder can be checked using verification. You may mix different "
+                    + "directory separators in this property. If so NiFi will unify all of them and will use windows's"
+                    + "directory separator: '\\' ")
+            .required(false)
+            .addValidator(NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MINIMUM_AGE = new PropertyDescriptor.Builder()
+            .displayName("Minimum file age in milliseconds")

Review Comment:
   Please use `StandardValidators.TIME_PERIOD_VALIDATOR` for time periods with arbitrary time units instead of milliseconds only.



##########
nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/ListSmb.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.smb;
+
+import static java.time.format.DateTimeFormatter.ISO_DATE_TIME;
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableMap;
+import static org.apache.nifi.components.state.Scope.CLUSTER;
+import static org.apache.nifi.processor.util.StandardValidators.INTEGER_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR;
+
+import java.io.IOException;
+import java.net.URI;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.services.smb.SmbConnectionPoolService;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"microsoft", "storage", "samba"})
+@SeeAlso({PutSmbFile.class, GetSmbFile.class})
+@CapabilityDescription("Retrieves a listing of files shared via SMB protocol. For each file that is listed, " +
+        "creates a FlowFile that represents the file. This Processor is designed to run on Primary Node only in " +
+        "a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left " +
+        "off without duplicating all of the data.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({
+        @WritesAttribute(attribute = "filename", description = "The name of the file that was read from filesystem."),
+        @WritesAttribute(attribute = "path", description =
+                "The path is set to the relative path of the file's directory "
+                        + "on filesystem compared to the Share and Input Directory properties and the configured host "
+                        + "and port inherited from the configured connection pool controller service. For example, for "
+                        + "a given remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listed from "
+                        + "smb://HOSTNAME:PORT/SHARE:DIRECTORY\\sub\\folder\\file then the path attribute will be set to \"sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "absolute.path", description =
+                "The absolute.path is set to the absolute path of the file's directory on the remote location. For example, "
+                        + "given a remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listen from "
+                        + "SHARE:\\DIRECTORY\\sub\\folder\\file then the absolute.path attribute will be set to "
+                        + "\"SHARE:\\DIRECTORY\\sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "identifier", description =
+                "The identifier of the file. This equals to the path attribute so two files with the same relative path "
+                        + "coming from different file shares considered to be identical."),
+        @WritesAttribute(attribute = "timestamp", description =
+                "The timestamp of when the file in the filesystem was modified as 'yyyy-MM-dd'T'HH:mm:ssZ'"),
+        @WritesAttribute(attribute = "size", description = "The number of bytes in the source file"),
+})
+@Stateful(scopes = {Scope.CLUSTER}, description =
+        "After performing a listing of files, the state of the previous listing can be stored in order to list files "
+                + "continuously without duplication."
+)
+public class ListSmb extends AbstractListProcessor<SmbListableEntity> {
+
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+            .displayName("Input Directory")
+            .name("directory")
+            .description("The network folder to which files should be written. This is the remaining relative " +
+                    "after the hostname: smb://HOSTNAME:PORT/SHARE/[DIRECTORY]\\sub\\directories. It is also possible "
+                    + " to add subdirectories using this property. The given path on the remote file share must exists. "
+                    + "The existence of the remote folder can be checked using verification. You may mix different "
+                    + "directory separators in this property. If so NiFi will unify all of them and will use windows's"
+                    + "directory separator: '\\' ")
+            .required(false)
+            .addValidator(NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MINIMUM_AGE = new PropertyDescriptor.Builder()
+            .displayName("Minimum file age in milliseconds")
+            .name("min-age")
+            .description(
+                    "Any file younger the the given value will be omitted. Ideally this value should be greater then"
+                            + "the amount of time needed to perform a list.")
+            .required(true)
+            .addValidator(INTEGER_VALIDATOR)
+            .addValidator(NON_NEGATIVE_INTEGER_VALIDATOR)
+            .defaultValue("5000")
+            .build();
+
+    public static final PropertyDescriptor SMB_LISTING_STRATEGY = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(LISTING_STRATEGY)
+            .allowableValues(BY_ENTITIES, NO_TRACKING, BY_TIMESTAMPS)
+            .build();
+
+    public static final PropertyDescriptor SMB_CONNECTION_POOL_SERVICE = new Builder()
+            .name("smb-connection-pool-service")
+            .displayName("SMB Connection Pool Service")
+            .description("Specifies the SMB Connection Pool to use for creating SMB connections.")
+            .required(true)
+            .identifiesControllerService(SmbConnectionPoolService.class)
+            .build();
+
+    public static final PropertyDescriptor SKIP_FILES_WITH_SUFFIX = new Builder()
+            .name("file-name-suffix-filter")
+            .displayName("File name suffix filter")
+            .description("Files ends with the given suffix will be omitted. This is handy when writing large data into "
+                    + "temporary files and then moved to a final one. Please be advised that writing data into files "
+                    + "first is highly recommended when using Entity Tracking or Timestamp based listing strategies.")
+            .required(false)
+            .addValidator(NON_EMPTY_VALIDATOR)
+            .addValidator(new MustNotContainDirectorySeparatorsValidator())
+            .build();
+    public static final PropertyDescriptor SHARE = new PropertyDescriptor.Builder()
+            .displayName("Share")
+            .name("share")
+            .description("The network share to which files should be listed from. This is the \"first folder\"" +
+                    "after the hostname: smb://hostname:port\\[share]\\dir1\\dir2")
+            .required(false)
+            .addValidator(NON_BLANK_VALIDATOR)
+            .build();
+
+    private static final List<PropertyDescriptor> PROPERTIES = unmodifiableList(asList(
+            AbstractListProcessor.TARGET_SYSTEM_TIMESTAMP_PRECISION,
+            AbstractListProcessor.RECORD_WRITER,
+            ListedEntityTracker.TRACKING_STATE_CACHE,
+            ListedEntityTracker.TRACKING_TIME_WINDOW,
+            ListedEntityTracker.INITIAL_LISTING_TARGET,
+            SMB_LISTING_STRATEGY,
+            SMB_CONNECTION_POOL_SERVICE,
+            DIRECTORY,
+            MINIMUM_AGE,
+            SKIP_FILES_WITH_SUFFIX,
+            SHARE

Review Comment:
   Please order the properties from most important to least important.
   Suggested order:
   - SMB_LISTING_STRATEGY
   - SMB_CONNECTION_POOL_SERVICE
   - SHARE
   - DIRECTORY
   - AbstractListProcessor.RECORD_WRITER
   - SKIP_FILES_WITH_SUFFIX
   - MINIMUM_AGE
   - AbstractListProcessor.TARGET_SYSTEM_TIMESTAMP_PRECISION
   - ListedEntityTracker.TRACKING_STATE_CACHE
   - ListedEntityTracker.TRACKING_TIME_WINDOW
   - ListedEntityTracker.INITIAL_LISTING_TARGET
   



##########
nifi-nar-bundles/nifi-smb-bundle/nifi-smb-connection-pool-api/src/main/java/org/apache/nifi/services/smb/SmbConnectionPoolService.java:
##########
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.services.smb;
+
+import com.hierynomus.smbj.session.Session;
+import java.net.URI;
+import org.apache.nifi.controller.ControllerService;
+
+public interface SmbConnectionPoolService extends ControllerService {
+
+    /**
+     * Returns the identifier of the service location.
+     *
+     * @return the remote location
+     */
+    URI getServiceLocation();
+
+    /**
+     * Returns the active session to use.
+     *
+     * @return the session.
+     */
+    Session getSession();

Review Comment:
   Returning `Session` object from the controller service seems to me fine. However, in that case it should be renamed to `SmbSessionProvider` or something like that.



##########
nifi-nar-bundles/nifi-smb-bundle/nifi-smb-processors/src/main/java/org/apache/nifi/processors/smb/ListSmb.java:
##########
@@ -0,0 +1,304 @@
+/*
+ * 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.smb;
+
+import static java.time.format.DateTimeFormatter.ISO_DATE_TIME;
+import static java.util.Arrays.asList;
+import static java.util.Collections.emptyList;
+import static java.util.Collections.unmodifiableList;
+import static java.util.Collections.unmodifiableMap;
+import static org.apache.nifi.components.state.Scope.CLUSTER;
+import static org.apache.nifi.processor.util.StandardValidators.INTEGER_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_BLANK_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR;
+import static org.apache.nifi.processor.util.StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR;
+
+import java.io.IOException;
+import java.net.URI;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+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.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyDescriptor.Builder;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.components.Validator;
+import org.apache.nifi.components.state.Scope;
+import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.util.list.AbstractListProcessor;
+import org.apache.nifi.processor.util.list.ListedEntityTracker;
+import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.services.smb.SmbConnectionPoolService;
+
+@PrimaryNodeOnly
+@TriggerSerially
+@Tags({"microsoft", "storage", "samba"})
+@SeeAlso({PutSmbFile.class, GetSmbFile.class})
+@CapabilityDescription("Retrieves a listing of files shared via SMB protocol. For each file that is listed, " +
+        "creates a FlowFile that represents the file. This Processor is designed to run on Primary Node only in " +
+        "a cluster. If the primary node changes, the new Primary Node will pick up where the previous node left " +
+        "off without duplicating all of the data.")
+@InputRequirement(Requirement.INPUT_FORBIDDEN)
+@WritesAttributes({
+        @WritesAttribute(attribute = "filename", description = "The name of the file that was read from filesystem."),
+        @WritesAttribute(attribute = "path", description =
+                "The path is set to the relative path of the file's directory "
+                        + "on filesystem compared to the Share and Input Directory properties and the configured host "
+                        + "and port inherited from the configured connection pool controller service. For example, for "
+                        + "a given remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listed from "
+                        + "smb://HOSTNAME:PORT/SHARE:DIRECTORY\\sub\\folder\\file then the path attribute will be set to \"sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "absolute.path", description =
+                "The absolute.path is set to the absolute path of the file's directory on the remote location. For example, "
+                        + "given a remote location smb://HOSTNAME:PORT/SHARE:\\DIRECTORY, and a file is being listen from "
+                        + "SHARE:\\DIRECTORY\\sub\\folder\\file then the absolute.path attribute will be set to "
+                        + "\"SHARE:\\DIRECTORY\\sub\\folder\\file\"."),
+        @WritesAttribute(attribute = "identifier", description =
+                "The identifier of the file. This equals to the path attribute so two files with the same relative path "
+                        + "coming from different file shares considered to be identical."),
+        @WritesAttribute(attribute = "timestamp", description =
+                "The timestamp of when the file in the filesystem was modified as 'yyyy-MM-dd'T'HH:mm:ssZ'"),
+        @WritesAttribute(attribute = "size", description = "The number of bytes in the source file"),
+})
+@Stateful(scopes = {Scope.CLUSTER}, description =
+        "After performing a listing of files, the state of the previous listing can be stored in order to list files "
+                + "continuously without duplication."
+)
+public class ListSmb extends AbstractListProcessor<SmbListableEntity> {
+
+    public static final PropertyDescriptor DIRECTORY = new PropertyDescriptor.Builder()
+            .displayName("Input Directory")
+            .name("directory")
+            .description("The network folder to which files should be written. This is the remaining relative " +
+                    "after the hostname: smb://HOSTNAME:PORT/SHARE/[DIRECTORY]\\sub\\directories. It is also possible "
+                    + " to add subdirectories using this property. The given path on the remote file share must exists. "
+                    + "The existence of the remote folder can be checked using verification. You may mix different "
+                    + "directory separators in this property. If so NiFi will unify all of them and will use windows's"
+                    + "directory separator: '\\' ")
+            .required(false)
+            .addValidator(NON_BLANK_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor MINIMUM_AGE = new PropertyDescriptor.Builder()
+            .displayName("Minimum file age in milliseconds")
+            .name("min-age")
+            .description(
+                    "Any file younger the the given value will be omitted. Ideally this value should be greater then"
+                            + "the amount of time needed to perform a list.")
+            .required(true)
+            .addValidator(INTEGER_VALIDATOR)
+            .addValidator(NON_NEGATIVE_INTEGER_VALIDATOR)
+            .defaultValue("5000")
+            .build();
+
+    public static final PropertyDescriptor SMB_LISTING_STRATEGY = new PropertyDescriptor.Builder()
+            .fromPropertyDescriptor(LISTING_STRATEGY)
+            .allowableValues(BY_ENTITIES, NO_TRACKING, BY_TIMESTAMPS)
+            .build();
+
+    public static final PropertyDescriptor SMB_CONNECTION_POOL_SERVICE = new Builder()
+            .name("smb-connection-pool-service")
+            .displayName("SMB Connection Pool Service")
+            .description("Specifies the SMB Connection Pool to use for creating SMB connections.")
+            .required(true)
+            .identifiesControllerService(SmbConnectionPoolService.class)
+            .build();
+
+    public static final PropertyDescriptor SKIP_FILES_WITH_SUFFIX = new Builder()
+            .name("file-name-suffix-filter")
+            .displayName("File name suffix filter")

Review Comment:
   Please use title case for property labels: "File Name Suffix Filter"



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

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

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