Re: TaskManager deadlock on NetworkBufferPool
Please use this link. https://gist.github.com/imamitjain/5ab84c2d9eaf06615ad912506a08f7e2 On Thu, Apr 19, 2018 at 10:37 PM, Ted Yu wrote: > Amit: > Execution plan attachment didn't come through. > > Please consider using third party website for storing the plan. > > FYI > > On Thu, Apr 19, 2018 at 10:04 AM, Amit Jain wrote: > > > @Ufuk Please find execution plan in the attachment. > > > > @Nico Job is not making progress at all. This issue is happening > > randomly. Few of our jobs are working with only few MB of data and still, > > they are getting stuck even TM have 22G with 2 slots per TM. > > > > I've started using 1.5 and facing few issues which I'm communicating with > > community these days. However, this issue seems to be solved there :-) Do > > you guys have a timeline for 1.5 release? > > > > -- > > Thanks, > > Amit > > > > > > > > > > > > On Fri, Apr 6, 2018 at 10:40 PM, Ufuk Celebi wrote: > > > >> Hey Amit! > >> > >> Thanks for posting this here. I don't think it's an issue of the > >> buffer pool per se. Instead I think there are two potential causes > >> here: > >> > >> 1. The generated flow doesn't use blocking intermediate results for a > >> branching-joining flow. > >> => I think we can check it if you run and post the output of > >> `StreamExecutionEnvironment#getExecutionPlan()` here. Can you please > >> post the result of this here? > >> > >> 2. The blocking intermediate result is used but there is an issue with > >> the implementation of them. > >> => Depending on the output of 1, we can investigate this option. > >> > >> As Fabian mentioned, running this with a newer version of Flink will > >> be very helpful. If the problem still persists, it will also make it > >> more likely that the issue will be fixed faster. ;-) > >> > >> – Ufuk > >> > >> > >> On Fri, Apr 6, 2018 at 5:43 AM, Nico Kruber > >> wrote: > >> > I'm not aware of any changes regarding the blocking buffer pools > though. > >> > > >> > Is it really stuck or just making progress slowly? (You can check with > >> > the number or records sent/received in the Web UI) > >> > > >> > Anyway, this may also simply mean that the task is back-pressured > >> > depending on how the operators are wired together. In that case, all > >> > available buffers for that ResultPartition have been used (records > were > >> > serialized into them) and are now waiting on Netty to send or a > >> > SpillingSubpartition to spill data to disk. > >> > Please also check for warnings or errors in the affected TaskManager's > >> > log files. > >> > > >> > If you can reproduce the problem, could you try reducing your program > to > >> > a minimal working example and provide it here for further debugging? > >> > > >> > > >> > Thanks > >> > Nico > >> > > >> > On 04/04/18 23:00, Fabian Hueske wrote: > >> >> Hi Amit, > >> >> > >> >> The network stack has been redesigned for the upcoming Flink 1.5 > >> release. > >> >> The issue might have been fixed by that. > >> >> > >> >> There's already a first release candidate for Flink 1.5.0 available > >> [1]. > >> >> It would be great if you would have the chance to check if the bug is > >> still > >> >> present. > >> >> > >> >> Best, Fabian > >> >> > >> >> [1] > >> >> https://lists.apache.org/thread.html/a6b6fb1a42a975608fa8641 > >> c86df30b47f022985ade845f1f1ec542a@%3Cdev.flink.apache.org%3E > >> >> > >> >> 2018-04-04 20:23 GMT+02:00 Ted Yu : > >> >> > >> >>> I searched for 0x0005e28fe218 in the two files you attached > >> >>> to FLINK-2685 but didn't find any hit. > >> >>> > >> >>> Was this the same instance as the attachment to FLINK-2685 ? > >> >>> > >> >>> Thanks > >> >>> > >> >>> On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain > >> wrote: > >> >>> > >> +u...@flink.apache.org > >> > >> On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain > >> 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=0x7f9ea84d2fb0 nid=0x22fe in > >> Object.wait() > >> > [0x7f9ebf102000] > >> >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.reques
Re: TaskManager deadlock on NetworkBufferPool
Amit: Execution plan attachment didn't come through. Please consider using third party website for storing the plan. FYI On Thu, Apr 19, 2018 at 10:04 AM, Amit Jain wrote: > @Ufuk Please find execution plan in the attachment. > > @Nico Job is not making progress at all. This issue is happening > randomly. Few of our jobs are working with only few MB of data and still, > they are getting stuck even TM have 22G with 2 slots per TM. > > I've started using 1.5 and facing few issues which I'm communicating with > community these days. However, this issue seems to be solved there :-) Do > you guys have a timeline for 1.5 release? > > -- > Thanks, > Amit > > > > > > On Fri, Apr 6, 2018 at 10:40 PM, Ufuk Celebi wrote: > >> Hey Amit! >> >> Thanks for posting this here. I don't think it's an issue of the >> buffer pool per se. Instead I think there are two potential causes >> here: >> >> 1. The generated flow doesn't use blocking intermediate results for a >> branching-joining flow. >> => I think we can check it if you run and post the output of >> `StreamExecutionEnvironment#getExecutionPlan()` here. Can you please >> post the result of this here? >> >> 2. The blocking intermediate result is used but there is an issue with >> the implementation of them. >> => Depending on the output of 1, we can investigate this option. >> >> As Fabian mentioned, running this with a newer version of Flink will >> be very helpful. If the problem still persists, it will also make it >> more likely that the issue will be fixed faster. ;-) >> >> – Ufuk >> >> >> On Fri, Apr 6, 2018 at 5:43 AM, Nico Kruber >> wrote: >> > I'm not aware of any changes regarding the blocking buffer pools though. >> > >> > Is it really stuck or just making progress slowly? (You can check with >> > the number or records sent/received in the Web UI) >> > >> > Anyway, this may also simply mean that the task is back-pressured >> > depending on how the operators are wired together. In that case, all >> > available buffers for that ResultPartition have been used (records were >> > serialized into them) and are now waiting on Netty to send or a >> > SpillingSubpartition to spill data to disk. >> > Please also check for warnings or errors in the affected TaskManager's >> > log files. >> > >> > If you can reproduce the problem, could you try reducing your program to >> > a minimal working example and provide it here for further debugging? >> > >> > >> > Thanks >> > Nico >> > >> > On 04/04/18 23:00, Fabian Hueske wrote: >> >> Hi Amit, >> >> >> >> The network stack has been redesigned for the upcoming Flink 1.5 >> release. >> >> The issue might have been fixed by that. >> >> >> >> There's already a first release candidate for Flink 1.5.0 available >> [1]. >> >> It would be great if you would have the chance to check if the bug is >> still >> >> present. >> >> >> >> Best, Fabian >> >> >> >> [1] >> >> https://lists.apache.org/thread.html/a6b6fb1a42a975608fa8641 >> c86df30b47f022985ade845f1f1ec542a@%3Cdev.flink.apache.org%3E >> >> >> >> 2018-04-04 20:23 GMT+02:00 Ted Yu : >> >> >> >>> I searched for 0x0005e28fe218 in the two files you attached >> >>> to FLINK-2685 but didn't find any hit. >> >>> >> >>> Was this the same instance as the attachment to FLINK-2685 ? >> >>> >> >>> Thanks >> >>> >> >>> On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain >> wrote: >> >>> >> +u...@flink.apache.org >> >> On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain >> 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=0x7f9ea84d2fb0 nid=0x22fe in >> Object.wait() >> > [0x7f9ebf102000] >> >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 <0x0005e28fe218> (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 <0x0005e29125f0> (a >> > org.apache.flink.r
Re: TaskManager deadlock on NetworkBufferPool
@Ufuk Please find execution plan in the attachment. @Nico Job is not making progress at all. This issue is happening randomly. Few of our jobs are working with only few MB of data and still, they are getting stuck even TM have 22G with 2 slots per TM. I've started using 1.5 and facing few issues which I'm communicating with community these days. However, this issue seems to be solved there :-) Do you guys have a timeline for 1.5 release? -- Thanks, Amit On Fri, Apr 6, 2018 at 10:40 PM, Ufuk Celebi wrote: > Hey Amit! > > Thanks for posting this here. I don't think it's an issue of the > buffer pool per se. Instead I think there are two potential causes > here: > > 1. The generated flow doesn't use blocking intermediate results for a > branching-joining flow. > => I think we can check it if you run and post the output of > `StreamExecutionEnvironment#getExecutionPlan()` here. Can you please > post the result of this here? > > 2. The blocking intermediate result is used but there is an issue with > the implementation of them. > => Depending on the output of 1, we can investigate this option. > > As Fabian mentioned, running this with a newer version of Flink will > be very helpful. If the problem still persists, it will also make it > more likely that the issue will be fixed faster. ;-) > > – Ufuk > > > On Fri, Apr 6, 2018 at 5:43 AM, Nico Kruber > wrote: > > I'm not aware of any changes regarding the blocking buffer pools though. > > > > Is it really stuck or just making progress slowly? (You can check with > > the number or records sent/received in the Web UI) > > > > Anyway, this may also simply mean that the task is back-pressured > > depending on how the operators are wired together. In that case, all > > available buffers for that ResultPartition have been used (records were > > serialized into them) and are now waiting on Netty to send or a > > SpillingSubpartition to spill data to disk. > > Please also check for warnings or errors in the affected TaskManager's > > log files. > > > > If you can reproduce the problem, could you try reducing your program to > > a minimal working example and provide it here for further debugging? > > > > > > Thanks > > Nico > > > > On 04/04/18 23:00, Fabian Hueske wrote: > >> Hi Amit, > >> > >> The network stack has been redesigned for the upcoming Flink 1.5 > release. > >> The issue might have been fixed by that. > >> > >> There's already a first release candidate for Flink 1.5.0 available [1]. > >> It would be great if you would have the chance to check if the bug is > still > >> present. > >> > >> Best, Fabian > >> > >> [1] > >> https://lists.apache.org/thread.html/a6b6fb1a42a975608fa8641c86df30 > b47f022985ade845f1f1ec542a@%3Cdev.flink.apache.org%3E > >> > >> 2018-04-04 20:23 GMT+02:00 Ted Yu : > >> > >>> I searched for 0x0005e28fe218 in the two files you attached > >>> to FLINK-2685 but didn't find any hit. > >>> > >>> Was this the same instance as the attachment to FLINK-2685 ? > >>> > >>> Thanks > >>> > >>> On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain wrote: > >>> > +u...@flink.apache.org > > On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain > 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=0x7f9ea84d2fb0 nid=0x22fe in > Object.wait() > > [0x7f9ebf102000] > >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 <0x0005e28fe218> (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 <0x0005e29125f0> (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.fli
Re: TaskManager deadlock on NetworkBufferPool
Hey Amit! Thanks for posting this here. I don't think it's an issue of the buffer pool per se. Instead I think there are two potential causes here: 1. The generated flow doesn't use blocking intermediate results for a branching-joining flow. => I think we can check it if you run and post the output of `StreamExecutionEnvironment#getExecutionPlan()` here. Can you please post the result of this here? 2. The blocking intermediate result is used but there is an issue with the implementation of them. => Depending on the output of 1, we can investigate this option. As Fabian mentioned, running this with a newer version of Flink will be very helpful. If the problem still persists, it will also make it more likely that the issue will be fixed faster. ;-) – Ufuk On Fri, Apr 6, 2018 at 5:43 AM, Nico Kruber wrote: > I'm not aware of any changes regarding the blocking buffer pools though. > > Is it really stuck or just making progress slowly? (You can check with > the number or records sent/received in the Web UI) > > Anyway, this may also simply mean that the task is back-pressured > depending on how the operators are wired together. In that case, all > available buffers for that ResultPartition have been used (records were > serialized into them) and are now waiting on Netty to send or a > SpillingSubpartition to spill data to disk. > Please also check for warnings or errors in the affected TaskManager's > log files. > > If you can reproduce the problem, could you try reducing your program to > a minimal working example and provide it here for further debugging? > > > Thanks > Nico > > On 04/04/18 23:00, Fabian Hueske wrote: >> Hi Amit, >> >> The network stack has been redesigned for the upcoming Flink 1.5 release. >> The issue might have been fixed by that. >> >> There's already a first release candidate for Flink 1.5.0 available [1]. >> It would be great if you would have the chance to check if the bug is still >> present. >> >> Best, Fabian >> >> [1] >> https://lists.apache.org/thread.html/a6b6fb1a42a975608fa8641c86df30b47f022985ade845f1f1ec542a@%3Cdev.flink.apache.org%3E >> >> 2018-04-04 20:23 GMT+02:00 Ted Yu : >> >>> I searched for 0x0005e28fe218 in the two files you attached >>> to FLINK-2685 but didn't find any hit. >>> >>> Was this the same instance as the attachment to FLINK-2685 ? >>> >>> Thanks >>> >>> On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain wrote: >>> +u...@flink.apache.org On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain 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=0x7f9ea84d2fb0 nid=0x22fe in Object.wait() > [0x7f9ebf102000] >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 <0x0005e28fe218> (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 <0x0005e29125f0> (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.Threa
Re: TaskManager deadlock on NetworkBufferPool
I'm not aware of any changes regarding the blocking buffer pools though. Is it really stuck or just making progress slowly? (You can check with the number or records sent/received in the Web UI) Anyway, this may also simply mean that the task is back-pressured depending on how the operators are wired together. In that case, all available buffers for that ResultPartition have been used (records were serialized into them) and are now waiting on Netty to send or a SpillingSubpartition to spill data to disk. Please also check for warnings or errors in the affected TaskManager's log files. If you can reproduce the problem, could you try reducing your program to a minimal working example and provide it here for further debugging? Thanks Nico On 04/04/18 23:00, Fabian Hueske wrote: > Hi Amit, > > The network stack has been redesigned for the upcoming Flink 1.5 release. > The issue might have been fixed by that. > > There's already a first release candidate for Flink 1.5.0 available [1]. > It would be great if you would have the chance to check if the bug is still > present. > > Best, Fabian > > [1] > https://lists.apache.org/thread.html/a6b6fb1a42a975608fa8641c86df30b47f022985ade845f1f1ec542a@%3Cdev.flink.apache.org%3E > > 2018-04-04 20:23 GMT+02:00 Ted Yu : > >> I searched for 0x0005e28fe218 in the two files you attached >> to FLINK-2685 but didn't find any hit. >> >> Was this the same instance as the attachment to FLINK-2685 ? >> >> Thanks >> >> On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain wrote: >> >>> +u...@flink.apache.org >>> >>> On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain 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=0x7f9ea84d2fb0 nid=0x22fe in >>> Object.wait() [0x7f9ebf102000] 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 <0x0005e28fe218> (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 <0x0005e29125f0> (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 >>> >> > -- Nico Kruber | Software Engineer data Artisans Follow us @dataArtisans -- Join Flink Forward - The Apache Flink Conference Stream Processing | Event Driven | Real Time -- Data Artisans GmbH | Stresemannstr. 121A,10963 Berlin, Germany data Artisans, Inc. | 1161 Mission Street, San Francisco, CA-94103, USA -- Data Artisans GmbH Registered at Amtsgericht Charlottenburg: HRB 158244 B Managing Directors: Dr. Kostas Tzoumas, Dr. Stephan Ewen signature.asc Description: OpenPGP digital signature
Re: TaskManager deadlock on NetworkBufferPool
Hi Amit, The network stack has been redesigned for the upcoming Flink 1.5 release. The issue might have been fixed by that. There's already a first release candidate for Flink 1.5.0 available [1]. It would be great if you would have the chance to check if the bug is still present. Best, Fabian [1] https://lists.apache.org/thread.html/a6b6fb1a42a975608fa8641c86df30b47f022985ade845f1f1ec542a@%3Cdev.flink.apache.org%3E 2018-04-04 20:23 GMT+02:00 Ted Yu : > I searched for 0x0005e28fe218 in the two files you attached > to FLINK-2685 but didn't find any hit. > > Was this the same instance as the attachment to FLINK-2685 ? > > Thanks > > On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain wrote: > > > +u...@flink.apache.org > > > > On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain 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=0x7f9ea84d2fb0 nid=0x22fe in > > Object.wait() > > > [0x7f9ebf102000] > > >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 <0x0005e28fe218> (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 <0x0005e29125f0> (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 > > >
Re: TaskManager deadlock on NetworkBufferPool
I searched for 0x0005e28fe218 in the two files you attached to FLINK-2685 but didn't find any hit. Was this the same instance as the attachment to FLINK-2685 ? Thanks On Wed, Apr 4, 2018 at 10:21 AM, Amit Jain wrote: > +u...@flink.apache.org > > On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain 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=0x7f9ea84d2fb0 nid=0x22fe in > Object.wait() > > [0x7f9ebf102000] > >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 <0x0005e28fe218> (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 <0x0005e29125f0> (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 >
Re: TaskManager deadlock on NetworkBufferPool
+u...@flink.apache.org On Wed, Apr 4, 2018 at 11:33 AM, Amit Jain 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=0x7f9ea84d2fb0 nid=0x22fe in Object.wait() > [0x7f9ebf102000] >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 <0x0005e28fe218> (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 <0x0005e29125f0> (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