[ 
https://issues.apache.org/jira/browse/NIFI-5604?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16686842#comment-16686842
 ] 

ASF GitHub Bot commented on NIFI-5604:
--------------------------------------

Github user patricker commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/3075#discussion_r233524977
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/GenerateTableFetch.java
 ---
    @@ -450,7 +450,33 @@ public void onTrigger(final ProcessContext context, 
final ProcessSessionFactory
     
                     // If there are no SQL statements to be generated, still 
output an empty flow file if specified by the user
                     if (numberOfFetches == 0 && 
outputEmptyFlowFileOnZeroResults) {
    -                    session.transfer((fileToProcess == null) ? 
session.create() : session.create(fileToProcess), REL_SUCCESS);
    +                    FlowFile emptyFlowFile = (fileToProcess == null) ? 
session.create() : session.create(fileToProcess);
    +                    Map<String, String> attributesToAdd = new HashMap<>();
    +
    +                    attributesToAdd.put("generatetablefetch.tableName", 
tableName);
    +                    if (columnNames != null) {
    +                        
attributesToAdd.put("generatetablefetch.columnNames", columnNames);
    +                    }
    +                    whereClause = maxValueClauses.isEmpty() ? "1=1" : 
StringUtils.join(maxValueClauses, " AND ");
    +                    if (StringUtils.isNotBlank(whereClause)) {
    +                        
attributesToAdd.put("generatetablefetch.whereClause", whereClause);
    +                    }
    +                    final String maxColumnNames = 
StringUtils.join(maxValueColumnNameList, ", ");
    +                    if (StringUtils.isNotBlank(maxColumnNames)) {
    +                        
attributesToAdd.put("generatetablefetch.maxColumnNames", maxColumnNames);
    +                    }
    +                    attributesToAdd.put("generatetablefetch.limit", null);
    --- End diff --
    
    There is no existing test in `master`, but I think in normal circumstances 
`limit` actually gets written as `"null"`.
    
    ```
    if ((i == numberOfFetches - 1) && useColumnValsForPaging && 
(maxValueClauses.isEmpty() || customWhereClause != null)) {
                                maxValueClauses.add(columnForPartitioning + " 
<= " + maxValueForPartitioning);
                                limit = null;
                            }
    ```
    
    This value is then written using `String.valueOf(limit)`, which is going to 
translate the `null` to `"null"`.


> Allow GenerateTableFetch to send empty flow files when no rows would be 
> fetched
> -------------------------------------------------------------------------------
>
>                 Key: NIFI-5604
>                 URL: https://issues.apache.org/jira/browse/NIFI-5604
>             Project: Apache NiFi
>          Issue Type: Improvement
>          Components: Extensions
>            Reporter: Matt Burgess
>            Assignee: Matt Burgess
>            Priority: Major
>
> Currently, GenerateTableFetch will not output a flow file if there are no 
> rows to be fetched. However, it may be desired (especially with incoming flow 
> files) that a flow file be sent out even if GTF does not generate any SQL. 
> This capability, along with the fragment attributes from NIFI-5601, would 
> allow the user to handle this downstream.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to