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

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

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

    https://github.com/apache/nifi/pull/3112#discussion_r228781049
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ReplaceText.java
 ---
    @@ -297,16 +297,27 @@ public void onTrigger(final ProcessContext context, 
final ProcessSession session
             } catch (StackOverflowError e) {
                 // Some regular expressions can produce many matches on large 
input data size using recursive code
                 // do not log the StackOverflowError stack trace
    -            logger.info("Transferred {} to 'failure' due to {}", new 
Object[] {flowFile, e.toString()});
    -            session.transfer(flowFile, REL_FAILURE);
    +            sendToFailure(session, flowFile, logger, e);
    +            return;
    +        } catch (RuntimeException e) {
    --- End diff --
    
    Please use following code instead of using String.startWith:
    ```suggestion
            } catch 
(AttributeExpressionLanguageException|IllegalAttributeException e) {
    ```
    
    I assume the intent of using startWith is catching both 
AttributeExpressionLanguageException and IllegalAttributeException without 
adding nifi-expression-language dependency. But having explicit dependency is 
preferable as it's more maintainable.


> ReplaceText processor can stop processing data if it evaluates invalid 
> expressions
> ----------------------------------------------------------------------------------
>
>                 Key: NIFI-5761
>                 URL: https://issues.apache.org/jira/browse/NIFI-5761
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Core Framework
>    Affects Versions: 1.5.0, 1.7.1
>         Environment: ALL
>            Reporter: Gardella Juan Pablo
>            Priority: Major
>   Original Estimate: 3h
>  Remaining Estimate: 3h
>
> Given a flowfile body with nifi expression, when _ReplaceText_ processor 
> evaluates it and the expression throws an exception, the processor will 
> rollback the flowfile and keep trying to evaluate instead of send the 
> flowfile to _failure_ relationshipt.
> Discussion Thread: 
> http://apache-nifi-users-list.2361937.n4.nabble.com/ReplaceText-cannot-consume-messages-if-Regex-does-not-match-td5986.html



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

Reply via email to