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

Steven Youtsey commented on NIFI-15739:
---------------------------------------

The failure I get is not caused by the assertion on the Relationship.
Here's what I get...it's an exception in the run method.
java.lang.AssertionError: 
org.apache.nifi.processor.exception.FlowFileHandlingException: 
FlowFile[0,4565253178189206.mockFlowFile,12B] is not known in this session
at 
org.apache.nifi.util.StandardProcessorTestRunner.run(StandardProcessorTestRunner.java:254)
at 
org.apache.nifi.util.StandardProcessorTestRunner.run(StandardProcessorTestRunner.java:212)
at 
org.apache.nifi.util.StandardProcessorTestRunner.run(StandardProcessorTestRunner.java:207)
at 
org.apache.nifi.util.StandardProcessorTestRunner.run(StandardProcessorTestRunner.java:202)
at 
org.apache.nifi.util.StandardProcessorTestRunner.run(StandardProcessorTestRunner.java:197)
at 
org.apache.nifi.processors.groovyx.ExecuteGroovyScriptTest.validExpressionLanguageInDynamicPropertyWithVariableValueWhichCannotBeUsedWithELExpression(ExecuteGroovyScriptTest.java:582)
Caused by: org.apache.nifi.processor.exception.FlowFileHandlingException: 
FlowFile[0,4565253178189206.mockFlowFile,12B] is not known in this session
at 
org.apache.nifi.util.MockProcessSession.validateState(MockProcessSession.java:1159)
at org.apache.nifi.util.MockProcessSession.remove(MockProcessSession.java:743)
at 
org.apache.nifi.processors.groovyx.flow.ProcessSessionWrap.revertReceivedTo(ProcessSessionWrap.java:211)
at 
org.apache.nifi.processors.groovyx.ExecuteGroovyScript.onTrigger(ExecuteGroovyScript.java:523)
at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
at 
org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:307)
at 
org.apache.nifi.util.StandardProcessorTestRunner$RunProcessor.call(StandardProcessorTestRunner.java:301)
at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:317)
at 
java.base/java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1144)
at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:642)
at java.base/java.lang.Thread.run(Thread.java:1583)
 
 
If you take the other approach I mentioned above, you'll see this same 
FFHandlingException. 
 
BTW: This is also what we see on our 2.9.0 systems.

> ExecuteGroovyScript: FlowFile not transferred to failure when dynamic 
> property contains Expression Language syntax error
> ------------------------------------------------------------------------------------------------------------------------
>
>                 Key: NIFI-15739
>                 URL: https://issues.apache.org/jira/browse/NIFI-15739
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 2.5.0
>            Reporter: Xinyu Wang
>            Assignee: Daniel Stieglitz
>            Priority: Major
>             Fix For: 2.9.0
>
>         Attachments: image-2026-03-25-08-33-20-522.png
>
>          Time Spent: 40m
>  Remaining Estimate: 0h
>
> When an ExecuteGroovyScript processor has a dynamic property whose value 
> contains an Expression Language syntax error, and the Failure Strategy is set 
> to "transfer to failure", the FlowFile in the upstream queue is not 
> transferred to the failure relationship. Instead, the FlowFile remains stuck 
> in the incoming queue, and the processor enters an infinite error loop — 
> logging the same error on every scheduling trigger.
> *Root Cause*
>   In ExecuteGroovyScript.onTrigger(), the execution order is:
>  # Evaluate dynamic properties — iterates context.getProperties() and calls 
> evaluateAttributeExpressions() on each dynamic property value
>  # Execute script — where session.get() is typically called by user code to 
> dequeue a FlowFile
>   The GroovyProcessSessionWrap maintains a toFail list that is only populated 
> when session.get() is called.
>   When a dynamic property has an EL syntax error:
>  * The exception is thrown at step 1
>  * script.run() (step 2) is never reached
>  * session.get() is never called
>  * toFail list remains empty
>  * The catch block calls session.revertReceivedTo(REL_FAILURE, t), which 
> iterates over the empty toFail list — effectively a no-op
>  * The FlowFile was never dequeued, so it stays in the incoming connection
> *Steps to Reproduce*
>  # Create a flow: GenerateFlowFile → ExecuteGroovyScript
>  # In ExecuteGroovyScript, set Failure Strategy to "Transfer to failure"
>  # Add a dynamic property (e.g. myProp) with an invalid EL expression as 
> value, e.g. *_${myparam:isempty()}_*
>  # Add a simple script body
>  # Connect the failure relationship to a downstream processor (e.g. 
> LogAttribute)
>  # Start the flow
>   {*}Expected{*}: FlowFile is transferred to the failure relationship with 
> ERROR_MESSAGE and ERROR_STACKTRACE attributes.
>   {*}Actual{*}: FlowFile remains stuck in the incoming queue. The processor 
> logs the expression language error repeatedly on every trigger. Nothing is 
> ever routed to failure.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to