+user@flink.apache.org
On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain <aj201...@gmail.com> wrote: > Hi, > > We are hitting TaskManager deadlock on NetworkBufferPool bug in Flink 1.3.2. > We have set of ETL's merge jobs for a number of tables and stuck with above > issue randomly daily. > > I'm attaching the thread dump of JobManager and one of the Task Manager (T1) > running stuck job. > We also observed, sometimes new job scheduled on T1 progresses even another > job is stuck there. > > "CHAIN DataSource (at createInput(ExecutionEnvironment.java:553) > (org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat)) -> Map (Map > at main(MergeTableSecond.java:175)) -> Map (Key Extractor) (6/9)" #1501 > daemon prio=5 os_prio=0 tid=0x00007f9ea84d2fb0 nid=0x22fe in Object.wait() > [0x00007f9ebf102000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at java.lang.Object.wait(Native Method) > at > org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestBuffer(LocalBufferPool.java:224) > - locked <0x00000005e28fe218> (a java.util.ArrayDeque) > at > org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestBufferBlocking(LocalBufferPool.java:193) > at > org.apache.flink.runtime.io.network.api.writer.RecordWriter.sendToTarget(RecordWriter.java:132) > - locked <0x00000005e29125f0> (a > org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer) > at > org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:89) > at > org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:65) > at > org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35) > at > org.apache.flink.runtime.operators.chaining.ChainedMapDriver.collect(ChainedMapDriver.java:79) > at > org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35) > at > org.apache.flink.runtime.operators.chaining.ChainedMapDriver.collect(ChainedMapDriver.java:79) > at > org.apache.flink.runtime.operators.util.metrics.CountingCollector.collect(CountingCollector.java:35) > at > org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:168) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:702) > at java.lang.Thread.run(Thread.java:748) > > -- > Thanks, > Amit