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

zoucao commented on FLINK-5463:
-------------------------------

I found the same problem in Flink 1.11, because of disk broken, task hang on up 
to 5 hours,until the task was killed manually. Here is the stack:
{code:java}
//代码占位符

"join-with-req (272/375)" #613403 prio=5 os_prio=0 tid=0x00007fa519419000 
nid=0x213d8 runnable [0x00007fa511b7d000]
   java.lang.Thread.State: RUNNABLE
        at org.rocksdb.RocksDB.disposeInternal(Native Method)
        at org.rocksdb.RocksObject.disposeInternal(RocksObject.java:37)
        at 
org.rocksdb.AbstractImmutableNativeReference.close(AbstractImmutableNativeReference.java:57)
        at 
org.apache.flink.contrib.streaming.state.RocksDBWrapper.close(RocksDBWrapper.java:208)
        at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:251)
        at 
org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.dispose(RocksDBKeyedStateBackend.java:401)
        at 
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.lambda$dispose$1(StreamOperatorStateHandler.java:135)
        at 
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler$$Lambda$350/1698627823.close(Unknown
 Source)
        at 
org.apache.flink.shaded.guava18.com.google.common.io.Closer.close(Closer.java:214)
        at 
org.apache.flink.streaming.api.operators.StreamOperatorStateHandler.dispose(StreamOperatorStateHandler.java:137)
        at 
org.apache.flink.streaming.api.operators.AbstractStreamOperator.dispose(AbstractStreamOperator.java:359)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:726)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.cleanUpInvoke(StreamTask.java:659)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:565)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:752)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:569)
        at java.lang.Thread.run(Thread.java:748)
{code}

> RocksDB.disposeInternal does not react to interrupts, blocks task cancellation
> ------------------------------------------------------------------------------
>
>                 Key: FLINK-5463
>                 URL: https://issues.apache.org/jira/browse/FLINK-5463
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / State Backends
>    Affects Versions: 1.2.0, 1.9.0
>            Reporter: Robert Metzger
>            Priority: Minor
>              Labels: auto-deprioritized-major
>
> I'm using Flink 699f4b0.
> My Flink job is slow while cancelling because RockDB seems to be busy with 
> disposing its state:
> {code}
> 2017-01-11 18:48:23,315 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Triggering cancellation of task code 
> TriggerWindow(TumblingEventTimeWindows(4), 
> ListStateDescriptor{serializer=org.apache.flink.api.java.typeutils.runtime.TupleSerializer@2edcd071
> }, EventTimeTrigger(), WindowedStream.apply(AllWindowedStream.java:440)) 
> (1/1) (2accc6ca2727c4f7ec963318fbd237e9).
> 2017-01-11 18:48:53,318 WARN  org.apache.flink.runtime.taskmanager.Task       
>               - Task 'TriggerWindow(TumblingEventTimeWindows(4), 
> ListStateDescriptor{serializer=org.apache.flink.api.java.typeutils.runtime.TupleSerializer@2edcd071},
>  EventTimeTrigger(), Windowed
> Stream.apply(AllWindowedStream.java:440)) (1/1)' did not react to cancelling 
> signal, but is stuck in method:
>  org.rocksdb.RocksDB.disposeInternal(Native Method)
> org.rocksdb.RocksObject.disposeInternal(RocksObject.java:37)
> org.rocksdb.AbstractImmutableNativeReference.close(AbstractImmutableNativeReference.java:56)
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.dispose(RocksDBKeyedStateBackend.java:250)
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.dispose(AbstractStreamOperator.java:331)
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.dispose(AbstractUdfStreamOperator.java:169)
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.dispose(WindowOperator.java:273)
> org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:439)
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:340)
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:654)
> java.lang.Thread.run(Thread.java:745)
> 2017-01-11 18:48:53,319 WARN  org.apache.flink.runtime.taskmanager.Task       
>               - Task 'TriggerWindow(TumblingEventTimeWindows(4), 
> ListStateDescriptor{serializer=org.apache.flink.api.java.typeutils.runtime.TupleSerializer@2edcd071},
>  EventTimeTrigger(), WindowedStream.apply(AllWindowedStream.java:440)) (1/1)' 
> did not react to cancelling signal, but is stuck in method:
>  org.rocksdb.RocksDB.disposeInternal(Native Method)
> org.rocksdb.RocksObject.disposeInternal(RocksObject.java:37)
> org.rocksdb.AbstractImmutableNativeReference.close(AbstractImmutableNativeReference.java:56)
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.dispose(RocksDBKeyedStateBackend.java:250)
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.dispose(AbstractStreamOperator.java:331)
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.dispose(AbstractUdfStreamOperator.java:169)
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.dispose(WindowOperator.java:273)
> org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:439)
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:340)
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:654)
> java.lang.Thread.run(Thread.java:745)
> 2017-01-11 18:49:23,319 WARN  org.apache.flink.runtime.taskmanager.Task       
>               - Task 'TriggerWindow(TumblingEventTimeWindows(4), 
> ListStateDescriptor{serializer=org.apache.flink.api.java.typeutils.runtime.TupleSerializer@2edcd071},
>  EventTimeTrigger(), WindowedStream.apply(AllWindowedStream.java:440)) (1/1)' 
> did not react to cancelling signal, but is stuck in method:
>  org.rocksdb.RocksDB.disposeInternal(Native Method)
> org.rocksdb.RocksObject.disposeInternal(RocksObject.java:37)
> org.rocksdb.AbstractImmutableNativeReference.close(AbstractImmutableNativeReference.java:56)
> org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.dispose(RocksDBKeyedStateBackend.java:250)
> org.apache.flink.streaming.api.operators.AbstractStreamOperator.dispose(AbstractStreamOperator.java:331)
> org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.dispose(AbstractUdfStreamOperator.java:169)
> org.apache.flink.streaming.runtime.operators.windowing.WindowOperator.dispose(WindowOperator.java:273)
> org.apache.flink.streaming.runtime.tasks.StreamTask.disposeAllOperators(StreamTask.java:439)
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:340)
> org.apache.flink.runtime.taskmanager.Task.run(Task.java:654)
> java.lang.Thread.run(Thread.java:745)
> 2017-01-11 18:49:50,080 INFO  org.apache.flink.runtime.taskmanager.Task       
>               - Freeing task resources for 
> TriggerWindow(TumblingEventTimeWindows(4), 
> ListStateDescriptor{serializer=org.apache.flink.api.java.typeutils.runtime.TupleSerializer@2edcd071},
>  EventTimeTrigger(), WindowedStream.apply(AllWindowedStream.java:440)) (1/1) 
> (2accc6ca2727c4f7ec963318fbd237e9)
> {code}
> I'm filing this issue because I didn't see such a behavior in Flink 1.1. I 
> guess Flink's code should be well behaved when it comes to cancelling tasks.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to