You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by "Ganesh, B (Nokia - IN/Bangalore)" <b....@nokia.com> on 2020/11/18 04:32:31 UTC

ListSftp/ListFtp Enhancements

We have below requirement to implement in apache Nifi , could you please let me know your view .


  1.  CNFI ListSFTP/ListFTP does not have an option to choose which files to list first (older/newer)
In NIFI, ListSFTP/ListFTP processors supporting an option to choose which files to list first (older/newer) And also this new field should support NiFi Expression Language.
Use case is the following:
We have a flow that is listing and fetching files from an external environment that can have thousands of files to be collected. There is a need to choose (depending on the environment) if we want to collect older files first or newer files first.
We have a process group that we instance with a few set of variables via API, and one of them should be this Transfer New Files First (variable TransferNewFilesFirst)
With the proposed feature, the ListSFTP/ListFTP processors would have this new field and inherit the variable e.g. ${TransferNewFilesFirst} and use it for listing.
Solution:
Added new property descriptor - "Transfer New Files First" in the ListSFTP/FTP processor. If true - then list the latest files first, else - list the latest files at the end. And also supports expression language (AbstractListProcessor.java) and added the same property in the ListSFTP.java and ListFTP.java files.
// new property descriptor
public static final PropertyDescriptor TRANSFER_NEW_FILES_FIRST = new PropertyDescriptor.Builder()
        .name("transfer-new-files-first")
        .displayName("Transfer New Files First")
        .description("If true, will list the newest resource first")
        .required(true)
        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
        .defaultValue("true")
        .build();

// to list the files in ascending or descending order
final boolean fileOrderController = context.getProperty(TRANSFER_NEW_FILES_FIRST).evaluateAttributeExpressions().asBoolean();
                             for (Long timestamp : (fileOrderController?orderedEntries.descendingKeySet():orderedEntries.keySet())) {
                    List<T> entities = orderedEntries.get(timestamp);
                    if (timestamp.equals(lastProcessedLatestEntryTimestampMillis)) {
                       // Filter out previously processed entities.
                       entities = entities.stream().filter(entity -> !latestIdentifiersProcessed.contains(entity.getIdentifier())).collect(Collectors.toList());
                    }

                    for (T entity : entities) {
                        // Create the FlowFile for this path.
                        final Map<String, String> attributes = createAttributes(entity, context);
                        FlowFile flowFile = session.create();
                        flowFile = session.putAllAttributes(flowFile, attributes);
                        session.transfer(flowFile, REL_SUCCESS);
                        flowfilesCreated++;
                    }
                }
            }

2. CNFI ListSFTP/ListFTP does not support Express Language for Search Recursively field
In NIFI, ListSFTP/ListFTP processors supporting the NiFi Expression Language for the Search Recursively field
Use case is the following:
We have a process group that we instance with a few set of variables via API, and one of them is the Search Recursively (variable SearchRecursively)
With the proposed feature, the ListSFTP/ListFTP processors would inherit the variable e.g. ${SearchRecursively} and use it for listing.
Having the feature would allow to instance the process group with the required variables from the get-go without further reconfigurations, thus making it a bit more reusable.
Solution:
Added "BOOLEAN_VALIDATOR" and "ExpressionLanguageScope" in the "Search Recursively" property. (FileTransfer.java, ListFile.java, FTPTransfer.java, SFTPTransfer.java)
public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
        .name("Search Recursively")
        .description("If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories")
        .required(true)
        .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
        .build();

Thanks & Regards,
Ganesh.B


RE: ListSftp/ListFtp Enhancements

Posted by "Ganesh, B (Nokia - IN/Bangalore)" <b....@nokia.com>.
Adding the option to get newest files first or oldest files first ?

Thanks & Regards,
Ganesh.B

-----Original Message-----
From: Bryan Bende <bb...@gmail.com> 
Sent: Wednesday, November 18, 2020 10:26 PM
To: dev@nifi.apache.org
Subject: Re: ListSftp/ListFtp Enhancements

Regarding adding support for expression language...

You could use parameter contexts instead of variables, all properties can reference parameters by default so no need to update property descriptors, just set to #{SearchRecursively} to reference the parameter.

In my opinion this would be better than removing the use of allowableValues on the Search Recursively just to support expression language.

Parameters are introduced in nifi 1.10.0.

On Tue, Nov 17, 2020 at 11:32 PM Ganesh, B (Nokia - IN/Bangalore) <b....@nokia.com> wrote:
>
> We have below requirement to implement in apache Nifi , could you please let me know your view .
>
>
>   1.  CNFI ListSFTP/ListFTP does not have an option to choose which 
> files to list first (older/newer) In NIFI, ListSFTP/ListFTP processors supporting an option to choose which files to list first (older/newer) And also this new field should support NiFi Expression Language.
> Use case is the following:
> We have a flow that is listing and fetching files from an external environment that can have thousands of files to be collected. There is a need to choose (depending on the environment) if we want to collect older files first or newer files first.
> We have a process group that we instance with a few set of variables 
> via API, and one of them should be this Transfer New Files First (variable TransferNewFilesFirst) With the proposed feature, the ListSFTP/ListFTP processors would have this new field and inherit the variable e.g. ${TransferNewFilesFirst} and use it for listing.
> Solution:
> Added new property descriptor - "Transfer New Files First" in the ListSFTP/FTP processor. If true - then list the latest files first, else - list the latest files at the end. And also supports expression language (AbstractListProcessor.java) and added the same property in the ListSFTP.java and ListFTP.java files.
> // new property descriptor
> public static final PropertyDescriptor TRANSFER_NEW_FILES_FIRST = new PropertyDescriptor.Builder()
>         .name("transfer-new-files-first")
>         .displayName("Transfer New Files First")
>         .description("If true, will list the newest resource first")
>         .required(true)
>         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
>         .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
>         .defaultValue("true")
>         .build();
>
> // to list the files in ascending or descending order final boolean 
> fileOrderController = context.getProperty(TRANSFER_NEW_FILES_FIRST).evaluateAttributeExpressions().asBoolean();
>                              for (Long timestamp : (fileOrderController?orderedEntries.descendingKeySet():orderedEntries.keySet())) {
>                     List<T> entities = orderedEntries.get(timestamp);
>                     if (timestamp.equals(lastProcessedLatestEntryTimestampMillis)) {
>                        // Filter out previously processed entities.
>                        entities = entities.stream().filter(entity -> !latestIdentifiersProcessed.contains(entity.getIdentifier())).collect(Collectors.toList());
>                     }
>
>                     for (T entity : entities) {
>                         // Create the FlowFile for this path.
>                         final Map<String, String> attributes = createAttributes(entity, context);
>                         FlowFile flowFile = session.create();
>                         flowFile = session.putAllAttributes(flowFile, attributes);
>                         session.transfer(flowFile, REL_SUCCESS);
>                         flowfilesCreated++;
>                     }
>                 }
>             }
>
> 2. CNFI ListSFTP/ListFTP does not support Express Language for Search 
> Recursively field In NIFI, ListSFTP/ListFTP processors supporting the 
> NiFi Expression Language for the Search Recursively field Use case is the following:
> We have a process group that we instance with a few set of variables 
> via API, and one of them is the Search Recursively (variable SearchRecursively) With the proposed feature, the ListSFTP/ListFTP processors would inherit the variable e.g. ${SearchRecursively} and use it for listing.
> Having the feature would allow to instance the process group with the required variables from the get-go without further reconfigurations, thus making it a bit more reusable.
> Solution:
> Added "BOOLEAN_VALIDATOR" and "ExpressionLanguageScope" in the "Search 
> Recursively" property. (FileTransfer.java, ListFile.java, FTPTransfer.java, SFTPTransfer.java) public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
>         .name("Search Recursively")
>         .description("If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories")
>         .required(true)
>         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
>         .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
>         .build();
>
> Thanks & Regards,
> Ganesh.B
>

Re: ListSftp/ListFtp Enhancements

Posted by Bryan Bende <bb...@gmail.com>.
Regarding adding support for expression language...

You could use parameter contexts instead of variables, all properties
can reference parameters by default so no need to update property
descriptors, just set to #{SearchRecursively} to reference the
parameter.

In my opinion this would be better than removing the use of
allowableValues on the Search Recursively just to support expression
language.

Parameters are introduced in nifi 1.10.0.

On Tue, Nov 17, 2020 at 11:32 PM Ganesh, B (Nokia - IN/Bangalore)
<b....@nokia.com> wrote:
>
> We have below requirement to implement in apache Nifi , could you please let me know your view .
>
>
>   1.  CNFI ListSFTP/ListFTP does not have an option to choose which files to list first (older/newer)
> In NIFI, ListSFTP/ListFTP processors supporting an option to choose which files to list first (older/newer) And also this new field should support NiFi Expression Language.
> Use case is the following:
> We have a flow that is listing and fetching files from an external environment that can have thousands of files to be collected. There is a need to choose (depending on the environment) if we want to collect older files first or newer files first.
> We have a process group that we instance with a few set of variables via API, and one of them should be this Transfer New Files First (variable TransferNewFilesFirst)
> With the proposed feature, the ListSFTP/ListFTP processors would have this new field and inherit the variable e.g. ${TransferNewFilesFirst} and use it for listing.
> Solution:
> Added new property descriptor - "Transfer New Files First" in the ListSFTP/FTP processor. If true - then list the latest files first, else - list the latest files at the end. And also supports expression language (AbstractListProcessor.java) and added the same property in the ListSFTP.java and ListFTP.java files.
> // new property descriptor
> public static final PropertyDescriptor TRANSFER_NEW_FILES_FIRST = new PropertyDescriptor.Builder()
>         .name("transfer-new-files-first")
>         .displayName("Transfer New Files First")
>         .description("If true, will list the newest resource first")
>         .required(true)
>         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
>         .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
>         .defaultValue("true")
>         .build();
>
> // to list the files in ascending or descending order
> final boolean fileOrderController = context.getProperty(TRANSFER_NEW_FILES_FIRST).evaluateAttributeExpressions().asBoolean();
>                              for (Long timestamp : (fileOrderController?orderedEntries.descendingKeySet():orderedEntries.keySet())) {
>                     List<T> entities = orderedEntries.get(timestamp);
>                     if (timestamp.equals(lastProcessedLatestEntryTimestampMillis)) {
>                        // Filter out previously processed entities.
>                        entities = entities.stream().filter(entity -> !latestIdentifiersProcessed.contains(entity.getIdentifier())).collect(Collectors.toList());
>                     }
>
>                     for (T entity : entities) {
>                         // Create the FlowFile for this path.
>                         final Map<String, String> attributes = createAttributes(entity, context);
>                         FlowFile flowFile = session.create();
>                         flowFile = session.putAllAttributes(flowFile, attributes);
>                         session.transfer(flowFile, REL_SUCCESS);
>                         flowfilesCreated++;
>                     }
>                 }
>             }
>
> 2. CNFI ListSFTP/ListFTP does not support Express Language for Search Recursively field
> In NIFI, ListSFTP/ListFTP processors supporting the NiFi Expression Language for the Search Recursively field
> Use case is the following:
> We have a process group that we instance with a few set of variables via API, and one of them is the Search Recursively (variable SearchRecursively)
> With the proposed feature, the ListSFTP/ListFTP processors would inherit the variable e.g. ${SearchRecursively} and use it for listing.
> Having the feature would allow to instance the process group with the required variables from the get-go without further reconfigurations, thus making it a bit more reusable.
> Solution:
> Added "BOOLEAN_VALIDATOR" and "ExpressionLanguageScope" in the "Search Recursively" property. (FileTransfer.java, ListFile.java, FTPTransfer.java, SFTPTransfer.java)
> public static final PropertyDescriptor RECURSIVE_SEARCH = new PropertyDescriptor.Builder()
>         .name("Search Recursively")
>         .description("If true, will pull files from arbitrarily nested subdirectories; otherwise, will not traverse subdirectories")
>         .required(true)
>         .addValidator(StandardValidators.BOOLEAN_VALIDATOR)
>         .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
>         .build();
>
> Thanks & Regards,
> Ganesh.B
>