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

Isha Lamboo commented on NIFI-11677:
------------------------------------

>From the Developers mailing list:

-----Oorspronkelijk bericht-----
Van: Bryan Bende <[email protected]> 
Verzonden: maandag 12 juni 2023 16:10
Aan: [email protected]
Onderwerp: Re: DeleteHDFS behavior when idle

 

The processor has @TriggerWhenEmpty so it is going to keep executing regardless 
of whether the incoming queue has data or not. I believe this was done early on 
for some processors that used Kerberos in order to allow the processor to have 
a chance to renew the Kerberos ticket, however we since moved away from need to 
do this, so unless there is another reason for having that, I would think it 
can be removed.

 

> 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)

Reply via email to