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

Matt Burgess commented on NIFI-3255:
------------------------------------

To me, it looks like that's there to make sure there's no data loss when 
merging multiple flow files (including the original) back into the original. 
Instead, I'm wondering why this line:
{code:title=SplitText.java:305|borderStyle=solid}
splitFlowFile = processSession.merge( Arrays.asList(new FlowFile[] { 
headerFlowFile, splitFlowFile }), splitFlowFile);
{code}
is passing in the splitFlowFile as both a flow file to merge as well as the 
destination. Shouldn't the flow file to merge be sourceFlowFile? If not 
(because of the offset and length), then perhaps it should be cloned twice, 
once to get an "input" flow file of the correct offset and length (to be 
removed later), and once to get an output flow file. Or perhaps the output flow 
file could be a session.create(sourceFlowFile)?

> SplitText fails with IllegalArgumentException: Destination cannot be within 
> sources
> -----------------------------------------------------------------------------------
>
>                 Key: NIFI-3255
>                 URL: https://issues.apache.org/jira/browse/NIFI-3255
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 1.1.0, 1.1.1
>            Reporter: Koji Kawamura
>            Assignee: Oleg Zhurakousky
>            Priority: Critical
>
> If SplitText is configured with Header Line Count grater than 0, and input 
> flow file like below is passed, it fails with IllegalArgumentException:
> {code}
> header
> line1
> line2
> {code}
> Stacktrace:
> {code}
> 2016-12-27 16:41:51,016 WARN [Timer-Driven Process Thread-2] 
> o.a.n.c.t.ContinuallyRunProcessorTask java.lang.IllegalArgumentException: 
> Destination cannot be within sources
>         at 
> org.apache.nifi.controller.repository.StandardProcessSession.merge(StandardProcessSession.java:2217)
>  ~[nifi-framework-core-1.2.0-SNAPSHOT.jar:1.2.0-SNAPSHOT]
>         at 
> org.apache.nifi.controller.repository.StandardProcessSession.merge(StandardProcessSession.java:2209)
>  ~[nifi-framework-core-1.2.0-SNAPSHOT.jar:1.2.0-SNAPSHOT]
>         at 
> org.apache.nifi.processors.standard.SplitText.generateSplitFlowFiles(SplitText.java:305)
>  ~[na:na]
>         at 
> org.apache.nifi.processors.standard.SplitText.onTrigger(SplitText.java:253) 
> ~[na:na]
>         at 
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
>  ~[nifi-api-1.2.0-SNAPSHOT.jar:1
> {code}
> Marked as critical since this is a regression and breaks existing flow. With 
> 1.0, SplitText produces following output flow files to "split" relationship 
> with the same input flow file and processor configuration:
> {code}
> # output flow file 1
> header
> line1
> # output flow file 2
> header
> line 2
> {code}
> The related code block in the processor has been covered by unit tests, 
> however, since unit test uses MockProcessSession, it skips check conditions 
> in StandardProcessSession. MockProcessSession should be updated, too, in 
> order to catch this type of issue with Unit testing.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to