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

ASF GitHub Bot commented on NIFI-5024:
--------------------------------------

GitHub user nsanglar opened a pull request:

    https://github.com/apache/nifi/pull/2594

    NIFI-5024 Resolves deadlock in ExecuteStreamCommand processor

    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:
    - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [x] 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.
    
    - [x] Has your PR been rebased against the latest commit within the target 
branch (typically master)?
    
    - [x] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [x] Have you ensured that the full suite of tests is executed via mvn 
-Pcontrib-check clean install at the root nifi folder?
    - [x] Have you written or updated unit tests to verify your changes?
    - [x] 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/Sqooba/nifi nifi-5024

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/2594.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 #2594
    
----
commit cd7bc120e3d82a0aa392b84717e54e2f76ae7ae2
Author: Nicolas Sanglard <nicolas@...>
Date:   2018-03-29T05:59:03Z

    NIFI-5024 Resolves deadlock in ExecuteStreamCommand processor

----


> Deadlock in ExecuteStreamCommand processor
> ------------------------------------------
>
>                 Key: NIFI-5024
>                 URL: https://issues.apache.org/jira/browse/NIFI-5024
>             Project: Apache NiFi
>          Issue Type: Bug
>    Affects Versions: 1.3.0
>            Reporter: Nicolas Sanglard
>            Priority: Minor
>         Attachments: Screen Shot 2018-03-28 at 15.34.36.png, Screen Shot 
> 2018-03-28 at 15.36.02.png
>
>
> Whenever a process is producing too much output on stderr, the current 
> implementation will run into a deadlock between the JVM and the unix process 
> started by the ExecuteStreamCommand.
> This is a known issue that is fully described here: 
> [http://java-monitor.com/forum/showthread.php?t=4067]
> In short:
>  * If the process produces too much stderr that is not consumed by 
> ExecuteStreamCommand, it will block until data is read.
>  * The current processor implementation is reading from stderr only after 
> having called process.waitFor()
>  * Thus, the two processes are waiting for each other and fall into a deadlock
>  
>  
> The following setup will lead to a deadlock:
>  
> A jar containing the following Main application:
> {code:java}
> object Main extends App {
>   import scala.collection.JavaConverters._
>   val str = 
> Source.fromInputStream(this.getClass.getResourceAsStream("/1mb.txt")).mkString
>   System.err.println(str)
> }
> {code}
> The following NiFi Flow:
>  
> !Screen Shot 2018-03-28 at 15.34.36.png!
>  
> Configuration for ExecuteStreamCommand:
>  
> !Screen Shot 2018-03-28 at 15.36.02.png!
>  
> The script is simply containing a call to the jar: 
> {code:java}
> java -jar stderr.jar
> {code}
>  
> Once the processor calls the script, it appears as "processing" indefinitely 
> and can only be stopped by restarting NiFi.
>  
> I already have a running solution that I will publish as soon as possible.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to