[
https://issues.apache.org/jira/browse/NIFI-6271?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16881678#comment-16881678
]
ASF subversion and git services commented on NIFI-6271:
-------------------------------------------------------
Commit fa1ed16e2bfc389466941e3bfe36a7a6ff08403b in nifi's branch
refs/heads/master from avseq1234
[ https://gitbox.apache.org/repos/asf?p=nifi.git;h=fa1ed16 ]
NIFI-6271, fix issue that incoming flowfile attributes don't copy into output
flowfiles when Output Batch Size is set
NIFI-6271, fix incoming flowfile attributes don't copy into output flowfiles
when Output Batch Size is set
NIFI-6271, fix incoming flowfile attributes don't copy into output flowfiles
when Output Batch Size is set
replace getAttribute(uuid) with getAttribute(CoreAttributes.UUID.key()
fix checkstyle violation
This closes #3575.
Signed-off-by: Koji Kawamura <[email protected]>
> ExecuteSQL incoming flowfile attributes not copied into output flowfiles when
> Output Batch Size is set
> ------------------------------------------------------------------------------------------------------
>
> Key: NIFI-6271
> URL: https://issues.apache.org/jira/browse/NIFI-6271
> Project: Apache NiFi
> Issue Type: Bug
> Components: Core Framework
> Affects Versions: 1.9.2
> Reporter: Arnaud Rivero
> Priority: Major
> Labels: easyfix, features, usability
> Original Estimate: 0.5h
> Time Spent: 1h 50m
> Remaining Estimate: 0h
>
> When using the executeSQL and executeSQLRecord processors, we can use input
> flowfiles with a certain number of attributes. If we don't set the Output
> Batch Size, all these attributes are copied to the output flowfile. However,
> if we set it, only the flowfiles from the first batch will have the
> attributes copied to. The flowfiles in the following batches will only have
> the default attributes.
> h2. Root cause
> In the source code of the method _onTrigger_ in the class
> _AbstractExecuteSQL,_ we have the following piece of code that is supposed to
> create an output flowfile and copy the original attributes into it:
> {code:java}
> FlowFile resultSetFF;
> if (fileToProcess == null) {
> resultSetFF = session.create();
> } else {
> resultSetFF = session.create(fileToProcess);
> resultSetFF = session.putAllAttributes(resultSetFF,
> fileToProcess.getAttributes());
> }
> {code}
> However the fix for the issue NIFI-6040 introduced this snippet way below in
> the same method:
>
> {code:java}
> // If we've reached the batch size, send out the flow files
> if (outputBatchSize > 0 && resultSetFlowFiles.size() >= outputBatchSize) {
> session.transfer(resultSetFlowFiles, REL_SUCCESS);
> // Need to remove the original input file if it exists
> if (fileToProcess != null) {
> session.remove(fileToProcess);
> fileToProcess = null;
> }
> session.commit();
> resultSetFlowFiles.clear();
> }
> {code}
> As you can see, it sets the variable fileToProcess to null, preventing the
> flowfiles in the next batch to copy its attributes
>
> h2.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)