[ https://issues.apache.org/jira/browse/NIFI-4836?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16351903#comment-16351903 ]
ASF GitHub Bot commented on NIFI-4836: -------------------------------------- Github user MikeThomsen commented on a diff in the pull request: https://github.com/apache/nifi/pull/2447#discussion_r165860112 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java --- @@ -315,6 +334,12 @@ public void onTrigger(final ProcessContext context, final ProcessSessionFactory session.getProvenanceReporter().receive(fileToProcess, jdbcURL, stopWatch.getElapsed(TimeUnit.MILLISECONDS)); resultSetFlowFiles.add(fileToProcess); + // If we've reached the batch size, send out the flow files + if (outputBatchSize > 0 && resultSetFlowFiles.size() >= outputBatchSize) { + session.transfer(resultSetFlowFiles, REL_SUCCESS); + session.commit(); + resultSetFlowFiles.clear(); --- End diff -- I usually reinitialize the collection in cases like this. I think in the past I've had weird behavior from a collection when using clear() a lot on a lot of referenced objects. Not a needed change and maybe that's paranoia on my part based on weirdness from past projects. > Allow QueryDatabaseTables to send out batches of flow files while result set > is being processed > ----------------------------------------------------------------------------------------------- > > Key: NIFI-4836 > URL: https://issues.apache.org/jira/browse/NIFI-4836 > Project: Apache NiFi > Issue Type: Improvement > Components: Extensions > Reporter: Matt Burgess > Assignee: Matt Burgess > Priority: Major > > Currently QueryDatabaseTable (QDT) will not transfer the outgoing flowfiles > to the downstream relationship(s) until the entire result set has been > processed (regardless of whether Max Rows Per Flow File is set). This is so > the maxvalue.* and fragment.count attributes can be set correctly for each > flow file. > However for very large result sets, the initial fetch can take a long time, > and depending on the setting of Max Rows Per FlowFile, there could be a great > number of FlowFiles transferred downstream as a large burst at the end of QDT > execution. > It would be nice for the user to be able to choose to have FlowFiles be > transferred downstream while the result set is still being processed. This > alleviates the "large burst at the end" by replacing it with smaller output > batches during processing. The tradeoff will be that if an Output Batch Size > is set, then the maxvalue.* and fragment.count attributes will not be set on > the outgoing flow files. -- This message was sent by Atlassian JIRA (v7.6.3#76005)