[jira] [Commented] (NIFI-11677) DeleteHDFS runs tasks with empty incoming connection
[ https://issues.apache.org/jira/browse/NIFI-11677?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17793441#comment-17793441 ] Emilio Setiadarma commented on NIFI-11677: -- I think `@TriggerWhenEmpty` can be removed from `DeleteHDFS`. `DeleteHDFS` does have a call to `getUserGroupInformation()` which does all the Kerberos stuff. Will remove `@TriggerWhenEmpty` in a followup Jira. > 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 >Assignee: Pierre Villard >Priority: Minor > Fix For: 2.0.0-M1, 1.24.0 > > Time Spent: 20m > Remaining Estimate: 0h > > 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)
[jira] [Commented] (NIFI-11677) DeleteHDFS runs tasks with empty incoming connection
[ https://issues.apache.org/jira/browse/NIFI-11677?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17780384#comment-17780384 ] ASF subversion and git services commented on NIFI-11677: Commit b822c27fb5bf409cd47157da40cd02588bdfcd53 in nifi's branch refs/heads/support/nifi-1.x from Pierre Villard [ https://gitbox.apache.org/repos/asf?p=nifi.git;h=b822c27fb5 ] NIFI-11677 Removed non required yield in DeleteHDFS This closes #7937 Signed-off-by: David Handermann (cherry picked from commit 1d125e9906090eedb66a7f6f8f1176ac67e007bc) > 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 >Assignee: Pierre Villard >Priority: Minor > Time Spent: 20m > Remaining Estimate: 0h > > 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)
[jira] [Commented] (NIFI-11677) DeleteHDFS runs tasks with empty incoming connection
[ https://issues.apache.org/jira/browse/NIFI-11677?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17780382#comment-17780382 ] ASF subversion and git services commented on NIFI-11677: Commit 1d125e9906090eedb66a7f6f8f1176ac67e007bc in nifi's branch refs/heads/main from Pierre Villard [ https://gitbox.apache.org/repos/asf?p=nifi.git;h=1d125e9906 ] NIFI-11677 Removed non required yield in DeleteHDFS This closes #7937 Signed-off-by: David Handermann > 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 >Assignee: Pierre Villard >Priority: Minor > Time Spent: 20m > Remaining Estimate: 0h > > 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)
[jira] [Commented] (NIFI-11677) DeleteHDFS runs tasks with empty incoming connection
[ https://issues.apache.org/jira/browse/NIFI-11677?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=17731640#comment-17731640 ] Isha Lamboo commented on NIFI-11677: >From the Developers mailing list: -Oorspronkelijk bericht- Van: Bryan Bende Verzonden: maandag 12 juni 2023 16:10 Aan: d...@nifi.apache.org 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)