You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Robert Metzger (JIRA)" <ji...@apache.org> on 2017/01/11 20:32:16 UTC
[jira] [Created] (FLINK-5463) RocksDB.disposeInternal does not
react to interrupts, blocks task cancellation
Robert Metzger created FLINK-5463:
-------------------------------------
Summary: 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: State Backends, Checkpointing
Affects Versions: 1.2.0
Reporter: Robert Metzger
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
(v6.3.4#6332)