[
https://issues.apache.org/jira/browse/NIFI-4772?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16324410#comment-16324410
]
ASF GitHub Bot commented on NIFI-4772:
--------------------------------------
GitHub user markap14 opened a pull request:
https://github.com/apache/nifi/pull/2403
NIFI-4772: Refactored how the @OnScheduled methods of processors is i…
…nvoked/monitored. The new method does away with the two previously created
8-thread thread pools and just uses the Timer-Driven thread pool that is used
by other framework tasks.
Thank you for submitting a contribution to Apache NiFi.
In order to streamline the review of the contribution we ask you
to ensure the following steps have been taken:
### For all changes:
- [ ] Is there a JIRA ticket associated with this PR? Is it referenced
in the commit message?
- [ ] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number
you are trying to resolve? Pay particular attention to the hyphen "-" character.
- [ ] Has your PR been rebased against the latest commit within the target
branch (typically master)?
- [ ] Is your initial contribution a single, squashed commit?
### For code changes:
- [ ] Have you ensured that the full suite of tests is executed via mvn
-Pcontrib-check clean install at the root nifi folder?
- [ ] Have you written or updated unit tests to verify your changes?
- [ ] If adding new dependencies to the code, are these dependencies
licensed in a way that is compatible for inclusion under [ASF
2.0](http://www.apache.org/legal/resolved.html#category-a)?
- [ ] If applicable, have you updated the LICENSE file, including the main
LICENSE file under nifi-assembly?
- [ ] If applicable, have you updated the NOTICE file, including the main
NOTICE file found under nifi-assembly?
- [ ] If adding new Properties, have you added .displayName in addition to
.name (programmatic access) for each of the new properties?
### For documentation related changes:
- [ ] Have you ensured that format looks appropriate for the output in
which it is rendered?
### Note:
Please ensure that once the PR is submitted, you check travis-ci for build
issues and submit an update to your PR as soon as possible.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/markap14/nifi NIFI-4772
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/nifi/pull/2403.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #2403
----
commit c59645e0bc780ca3f2f997437902fe4e498f528d
Author: Mark Payne <markap14@...>
Date: 2018-01-12T19:12:57Z
NIFI-4772: Refactored how the @OnScheduled methods of processors is
invoked/monitored. The new method does away with the two previously created
8-thread thread pools and just uses the Timer-Driven thread pool that is used
by other framework tasks.
----
> If several processors do not return from their @OnScheduled method, NiFi will
> stop scheduling any Processors
> ------------------------------------------------------------------------------------------------------------
>
> Key: NIFI-4772
> URL: https://issues.apache.org/jira/browse/NIFI-4772
> Project: Apache NiFi
> Issue Type: Bug
> Components: Core Framework
> Reporter: Mark Payne
> Assignee: Mark Payne
> Priority: Critical
>
> If a Processor does not properly return from its @OnScheduled method and
> several instances of the processor are started, we can get into a state where
> no Processors can start. We start seeing log messages like the following:
> {code}
> 2018-01-10 10:16:31,433 WARN [StandardProcessScheduler Thread-1]
> o.a.n.controller.StandardProcessorNode Timed out while waiting for
> OnScheduled of 'UpdateAttribute' processor to finish. An attempt is made to
> cancel the task via Thread.interrupt(). However it does not guarantee that
> the task will be canceled since the code inside current OnScheduled operation
> may have been written to ignore interrupts which may result in a runaway
> thread. This could lead to more issues, eventually requiring NiFi to be
> restarted. This is usually a bug in the target Processor
> 'UpdateAttribute[id=95423ee6-e6a6-1220-83ad-af20577063bd]' that needs to be
> documented, reported and eventually fixed.
> 2018-01-10 10:16:42,937 WARN [StandardProcessScheduler Thread-2]
> o.a.n.controller.StandardProcessorNode Timed out while waiting for
> OnScheduled of 'PutHDFS' processor to finish. An attempt is made to cancel
> the task via Thread.interrupt(). However it does not guarantee that the task
> will be canceled since the code inside current OnScheduled operation may have
> been written to ignore interrupts which may result in a runaway thread. This
> could lead to more issues, eventually requiring NiFi to be restarted. This is
> usually a bug in the target Processor
> 'PutHDFS[id=25e531ec-d873-1dec-acc9-ea745e7869ed]' that needs to be
> documented, reported and eventually fixed.
> 2018-01-10 10:16:46,993 WARN [StandardProcessScheduler Thread-4]
> o.a.n.controller.StandardProcessorNode Timed out while waiting for
> OnScheduled of 'LogAttribute' processor to finish. An attempt is made to
> cancel the task via Thread.interrupt(). However it does not guarantee that
> the task will be canceled since the code inside current OnScheduled operation
> may have been written to ignore interrupts which may result in a runaway
> thread. This could lead to more issues, eventually requiring NiFi to be
> restarted. This is usually a bug in the target Processor
> 'LogAttribute[id=9a683a06-aa24-19b5-ffff-ffff944a0216]' that needs to be
> documented, reported and eventually fixed.
> {code}
> While we should avoid having misbehaving Processors to begin with, the
> framework must also be tolerant of this and should not allow one misbehaving
> Processor from affecting other Processors.
> We can "approximate" this issue by following these steps:
> 1. Create 1 DebugFlow Processor. Auto-terminate its success & failure
> relationships. Set the "@OnScheduled Pause Time" property to "2 mins"
> 2. Copy & paste this DebugFlow Processor so that there are at least 8 of them.
> 3. Create a GenerateFlowFile Processor and an UpdateAttribute Processor. Send
> success of GenerateFlowFile to UpdateAttribute.
> 4. Start all of the DebugFlow Processors.
> 5. Start the GenerateFlowFIle and UpdateAttribute Processors.
> In this scenario, we will not see the above log messages, because after 1
> minute the DebugFlow Processor is interrupted and the @OnSchedule method
> completes Exceptionally. However, we do see that GenerateFlowFile and
> UpdateAttribute do not start running until after the 2 minute time window has
> elapsed. If DebugFlow instead did not complete Exceptionally, then
> GenerateFlowFile and UpdateAttribute would never start running and we would
> see the above error messages in the log.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)