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

ASF GitHub Bot commented on FLINK-5465:
---------------------------------------

GitHub user StefanRRichter opened a pull request:

    https://github.com/apache/flink/pull/5058

    [FLINK-5465] [streaming] Wait for pending timer threads to finish or …

    …to exceed a time limit in exceptional stream task shutdown
    
    ## What is the purpose of the change
    
    When stream tasks are going through their cleanup in the failover case, 
pending timer threads can still access native resources of a state backend 
after the backend's disposal. In some cases, this can crash the JVM. 
    
    The obvious fix is to wait until all timers are finished before disposing 
those resources, and the main reason why we did not change this is that (in 
theory) a the user code of a triggering timer can block forever and suppress or 
delay restarts in failover cases. However, the situation has changed a bit 
since this topic was discussed for the last time and there is now also a 
watchdog that ensures that tasks are terminated after some time of inactivity. 
I would propose a middle ground that probably catches close to all of those 
problems, while still ensuring a reasonable fast shutdown. I would suggest to 
wait for the termination of the threadpool that runs all timer events until a 
time limit. Typically, there is a very limited number of in-flight timers and 
they are usually short lived. A wait interval of a few seconds should basically 
fix this problem, even though there can still be very rare cases of very long 
running timers that also happen to still access the disposed resources. But the 
likelihood of this scenario should be reduced by orders of magnitude and in 
particular, the cascading effect should be mitigated.
    
    
    ## Brief change log
    
      - *SteamTasks wait in their cleanup code for a certain time limit to give 
the timer service a chance to finish all pending timer threads. *
    
    ## Verifying this change
    
    This change added tests to `SystemProcessingTimeServiceTest`.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (no)
      - The public API, i.e., is any changed class annotated with 
`@Public(Evolving)`: (no)
      - The serializers: (no)
      - The runtime per-record code paths (performance sensitive): (no)
      - Anything that affects deployment or recovery: JobManager (and its 
components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes)
      - The S3 file system connector: (no)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (no)
      - If yes, how is the feature documented? (not applicable)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/StefanRRichter/flink 
FLINK-5465-wait-timer-shutdown

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

    https://github.com/apache/flink/pull/5058.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 #5058
    
----
commit 3c6f0f3377abb35ed5b04fdcdb599705bc0af162
Author: Stefan Richter <[email protected]>
Date:   2017-11-22T16:52:35Z

    [FLINK-5465] [streaming] Wait for pending timer threads to finish or to 
exceed a time limit in exceptional stream task shutdown

----


> RocksDB fails with segfault while calling AbstractRocksDBState.clear()
> ----------------------------------------------------------------------
>
>                 Key: FLINK-5465
>                 URL: https://issues.apache.org/jira/browse/FLINK-5465
>             Project: Flink
>          Issue Type: Bug
>          Components: State Backends, Checkpointing
>    Affects Versions: 1.2.0
>            Reporter: Robert Metzger
>         Attachments: hs-err-pid26662.log
>
>
> I'm using Flink 699f4b0.
> {code}
> #
> # A fatal error has been detected by the Java Runtime Environment:
> #
> #  SIGSEGV (0xb) at pc=0x00007f91a0d49b78, pid=26662, tid=140263356024576
> #
> # JRE version: Java(TM) SE Runtime Environment (7.0_67-b01) (build 
> 1.7.0_67-b01)
> # Java VM: Java HotSpot(TM) 64-Bit Server VM (24.65-b04 mixed mode 
> linux-amd64 compressed oops)
> # Problematic frame:
> # C  [librocksdbjni-linux64.so+0x1aeb78]  
> rocksdb::GetColumnFamilyID(rocksdb::ColumnFamilyHandle*)+0x8
> #
> # Failed to write core dump. Core dumps have been disabled. To enable core 
> dumping, try "ulimit -c unlimited" before starting Java again
> #
> # An error report file with more information is saved as:
> # 
> /yarn/nm/usercache/robert/appcache/application_1484132267957_0007/container_1484132267957_0007_01_000010/hs_err_pid26662.log
> Compiled method (nm) 1869778  903     n       org.rocksdb.RocksDB::remove 
> (native)
>  total in heap  [0x00007f91b40b9dd0,0x00007f91b40ba150] = 896
>  relocation     [0x00007f91b40b9ef0,0x00007f91b40b9f48] = 88
>  main code      [0x00007f91b40b9f60,0x00007f91b40ba150] = 496
> #
> # If you would like to submit a bug report, please visit:
> #   http://bugreport.sun.com/bugreport/crash.jsp
> # The crash happened outside the Java Virtual Machine in native code.
> # See problematic frame for where to report the bug.
> #
> {code}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to