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

Till Rohrmann commented on FLINK-12426:
---------------------------------------

Hmm, this sounds strange and should not happen. Even though not much of the 
code changed, it would be good to know whether the problem can be reproduced 
with Flink 1.8.0 as well.

To further debug the problem it would be good to decrease the number of TMs as 
much as possible and then share with us the logs on DEBUG log level. Maybe the 
contain some information on what is going wrong here.

> TM occasionally hang in deploying state
> ---------------------------------------
>
>                 Key: FLINK-12426
>                 URL: https://issues.apache.org/jira/browse/FLINK-12426
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>            Reporter: Qi
>            Priority: Major
>
> Hi all,
>   
>  We use Flink batch and start thousands of jobs per day. Occasionally we 
> observed some stuck jobs, due to some TM hang in “DEPLOYING” state. 
>   
>  It seems that the TM is calling BlobClient to download jars from 
> JM/BlobServer. Under hood it’s calling Socket.connect() and then 
> Socket.read() to retrieve results. 
>   
>  These jobs usually have many TM slots (1~2k). We checked the TM log and 
> dumped the TM thread. It indeed hung on socket read to download jar from Blob 
> server. 
>   
>  We're using Flink 1.5 but this may also affect later versions since related 
> code are not changed much. We've tried to add socket timeout in BlobClient, 
> but still no luck.
>   
>  ————————
>  TM log
>  ————————
>  ...
>  INFO org.apache.flink.runtime.taskexecutor.TaskExecutor - Received task 
> DataSource (at createInput(ExecutionEnvironment.java:548) (our.code)) 
> (184/2000).
> INFO org.apache.flink.runtime.taskmanager.Task - DataSource (at 
> createInput(ExecutionEnvironment.java:548) (our.code)) (184/2000) switched 
> from CREATED to DEPLOYING.
> INFO org.apache.flink.runtime.taskmanager.Task - Creating FileSystem stream 
> leak safety net for task DataSource (at 
> createInput(ExecutionEnvironment.java:548) (our.code)) (184/2000) [DEPLOYING]
> INFO org.apache.flink.runtime.taskmanager.Task - Loading JAR files for task 
> DataSource (at createInput(ExecutionEnvironment.java:548) (our.code)) 
> (184/2000) [DEPLOYING].
> INFO org.apache.flink.runtime.blob.BlobClient - Downloading 
> 19e65c0caa41f264f9ffe4ca2a48a434/p-3ecd6341bf97d5512b14c93f6c9f51f682b6db26-37d5e69d156ee00a924c1ebff0c0d280
>  from some-host-ip-port
> {color:#222222}no more logs...{color}
>   
>  ————————
>  TM thread dump:
>  ————————
>  _"DataSource (at createInput(ExecutionEnvironment.java:548) (our.code)) 
> (1999/2000)" #72 prio=5 os_prio=0 tid=0x00007fb9a1521000 nid=0xa0994 runnable 
> [0x00007fb97cfbf000]_
>     _java.lang.Thread.State: RUNNABLE_
>          _at java.net.SocketInputStream.socketRead0(Native Method)_
>          _at 
> java.net.SocketInputStream.socketRead(SocketInputStream.java:116)_
>          _at java.net.SocketInputStream.read(SocketInputStream.java:171)_
>          _at java.net.SocketInputStream.read(SocketInputStream.java:141)_
>          _at 
> org.apache.flink.runtime.blob.BlobInputStream.read(BlobInputStream.java:152)_
>          _at 
> org.apache.flink.runtime.blob.BlobInputStream.read(BlobInputStream.java:140)_
>          _at 
> org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:170)_
>          _at 
> org.apache.flink.runtime.blob.AbstractBlobCache.getFileInternal(AbstractBlobCache.java:181)_
>          _at 
> org.apache.flink.runtime.blob.PermanentBlobCache.getFile(PermanentBlobCache.java:206)_
>          _at 
> org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager.registerTask(BlobLibraryCacheManager.java:120)_
>          _- locked <0x000000078ab60ba8> (a java.lang.Object)_
>          _at 
> org.apache.flink.runtime.taskmanager.Task.createUserCodeClassloader(Task.java:893)_
>          _at org.apache.flink.runtime.taskmanager.Task.run(Task.java:585)_
>          _at java.lang.Thread.run(Thread.java:748)_
>  _————————_
>   



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to