You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "Isha Lamboo (Jira)" <ji...@apache.org> on 2023/06/12 14:29:00 UTC

[jira] [Updated] (NIFI-11677) DeleteHDFS runs tasks with empty incoming connection

     [ https://issues.apache.org/jira/browse/NIFI-11677?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Isha Lamboo updated NIFI-11677:
-------------------------------
    Summary: DeleteHDFS runs tasks with empty incoming connection  (was: DeleteHDFS is scheduled with empty incoming connection)

> DeleteHDFS runs tasks with empty incoming connection
> ----------------------------------------------------
>
>                 Key: NIFI-11677
>                 URL: https://issues.apache.org/jira/browse/NIFI-11677
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 1.18.0, 1.19.1
>         Environment: 64bit redhat linux, java 1.8.0_352, 3 node cluster
>            Reporter: Isha Lamboo
>            Priority: Minor
>
> The DeleteHDFS processor can operate either with or without an incoming connection. When operating *with* an incoming connection that has no flowfiles, it still registers as executing tasks and taking a small amount of time (in the range of 50-100 millis on our clusters). In our case this results in ~200 processors each running 800-1000 tasks per 5 minutes on a 3 node cluster with 16 CPUs per node.
> This is in contrast to the PutHDFS with an incoming connection which shows as 0 tasks/0 millis.
> Reproduce the situation by connecting a (stopped) ListHDFS processor to a DeleteHDFS processor with a variable for the path. Start the DeleteHDFS and refresh for statistics update.
> Having compared the code for PutHDFS and DeleteHDFS I think I see the cause:
> PutHDFS has this in its OnTrigger method:
> {code:java}
>         final FlowFile flowFile = session.get();
>         if (flowFile == null) {
>             return;
>         } {code}
> DeleteHDFS has this in the OnTrigger method:
> {code:java}
>         final FlowFile originalFlowFile = session.get();
>         // If this processor has an incoming connection, then do not run unless a
>         // FlowFile is actually sent through
>         if (originalFlowFile == null && context.hasIncomingConnection()) {
>             context.yield();
>             return;
>         } {code}
> I'm guessing that the extra context.yield() causes the framework to execute some administration tasks and update counters, where it does not for PutHDFS.
> I don't know how much overhead this causes, probably very little, but I'm trying to improve performance on a cluster with thread starvation issues and high CPU usage and these keep popping up in the summaries looking bad when I think they shouldn't.
> The likely solution is to only perform the context.yield() when hasIncomingConnection() returns false.



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