[
https://issues.apache.org/jira/browse/NIFI-14004?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Bob Paulin updated NIFI-14004:
------------------------------
Description:
I've hit a scenario with a merge processor that cause it to be executed over
and over again (See https://issues.apache.org/jira/browse/NIFI-14003) When
this happens the StandardStatelessFlowCurrent class continuously triggers [1]
the processor and for processors that extend BinFiles[2],
StatelessProcessSession objects are created and registered with
StandardExecutionProgress createSessions list until memory is exhausted
!NiFi-Stateless-OOM.png! .
These sessions appear to be saved in-memory to allow for rollback. Perhaps we
need a limit on the size of this list to prevent the OOM? Could we
automatically rollback if this limit is hit? Perhaps this is something
configurable to be passed in from the DataflowTriggerContext object when
StandardExecutionProgress is created [4]? If the transaction threshold is met
we will also continuously trigger[5] should there be a way to exit that as well?
[1][https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L273|https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L231]
[2][https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-extension-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java#L271]
[3]
[https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java#L108]
[4]
[https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java#L511]
[5]
[https://github.com/apache/nifi/blob/919b376c1622943be79cb0fbd6087d6221b5ac3e/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L253]
was:
I've hit a scenario with a merge processor that cause it to be executed over
and over again (See https://issues.apache.org/jira/browse/NIFI-14003) When
this happens the StandardStatelessFlowCurrent class continuously triggers [1]
the processor and for processors that extend BinFiles[2],
StatelessProcessSession objects are created and registered with
StandardExecutionProgress createSessions list until memory is exhausted
!NiFi-Stateless-OOM.png! .
These sessions appear to be saved in-memory to allow for rollback. Perhaps we
need a limit on the size of this list to prevent the OOM? Could we
automatically rollback if this limit is hit? Perhaps this is something
configurable to be passed in from the DataflowTriggerContext object when
StandardExecutionProgress is created [4]?
[1][https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L273|https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L231]
[2][https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-extension-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java#L271]
[3]
[https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java#L108]
[4]
[https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java#L511]
> NiFi Stateless Engine memory leak when execution gets stuck a single processor
> ------------------------------------------------------------------------------
>
> Key: NIFI-14004
> URL: https://issues.apache.org/jira/browse/NIFI-14004
> Project: Apache NiFi
> Issue Type: Bug
> Components: NiFi Stateless
> Reporter: Bob Paulin
> Priority: Major
> Attachments: NiFi-Stateless-OOM.png
>
>
> I've hit a scenario with a merge processor that cause it to be executed over
> and over again (See https://issues.apache.org/jira/browse/NIFI-14003) When
> this happens the StandardStatelessFlowCurrent class continuously triggers [1]
> the processor and for processors that extend BinFiles[2],
> StatelessProcessSession objects are created and registered with
> StandardExecutionProgress createSessions list until memory is exhausted
> !NiFi-Stateless-OOM.png! .
> These sessions appear to be saved in-memory to allow for rollback. Perhaps
> we need a limit on the size of this list to prevent the OOM? Could we
> automatically rollback if this limit is hit? Perhaps this is something
> configurable to be passed in from the DataflowTriggerContext object when
> StandardExecutionProgress is created [4]? If the transaction threshold is
> met we will also continuously trigger[5] should there be a way to exit that
> as well?
>
> [1][https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L273|https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L231]
> [2][https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-extension-bundles/nifi-extension-utils/nifi-bin-manager/src/main/java/org/apache/nifi/processor/util/bin/BinFiles.java#L271]
> [3]
> [https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/engine/StandardExecutionProgress.java#L108]
> [4]
> [https://github.com/apache/nifi/blob/6c4ddf86310187e093070446412c4b2314bd324a/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlow.java#L511]
> [5]
> [https://github.com/apache/nifi/blob/919b376c1622943be79cb0fbd6087d6221b5ac3e/nifi-stateless/nifi-stateless-bundle/nifi-stateless-engine/src/main/java/org/apache/nifi/stateless/flow/StandardStatelessFlowCurrent.java#L253]
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
