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

Dawid Wysakowicz commented on FLINK-10213:
------------------------------------------

Hi [~fwiffo],
Does it make sense for you what I wrote in my previous comment?

> Task managers cache a negative DNS lookup of the blob server indefinitely
> -------------------------------------------------------------------------
>
>                 Key: FLINK-10213
>                 URL: https://issues.apache.org/jira/browse/FLINK-10213
>             Project: Flink
>          Issue Type: Bug
>          Components: TaskManager
>    Affects Versions: 1.5.0
>            Reporter: Joey Echeverria
>            Assignee: Joey Echeverria
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.8.0
>
>
> When the task manager establishes a connection with the resource manager, it 
> gets the hostname and port of the blob server and uses that to create an 
> instance of an {{InetSocketAddress}}. Per the documentation of the 
> constructor:
> {quote}An attempt will be made to resolve the hostname into an InetAddress. 
> If that attempt fails, the address will be flagged as _unresolved_{quote}
> Flink never checks to see if the address was unresolved. Later when executing 
> a task that needs to download from the blob server, it will use that same 
> {{InetSocketAddress}} instance to attempt to connect a {{Socket}}. This will 
> result in an exception similar to:
> {noformat}
> java.io.IOException: Failed to fetch BLOB 
> 97799b827ef073e04178a99f0f40b00e/p-6d8ec2ad31337110819c7c3641fdb18d3793a7fb-72bf00066308f4b4d2a9c5aea593b41f
>  from jobmanager:6124 and store it under 
> /tmp/blobStore-d135961a-03cb-4542-af6d-cf378ff83c12/incoming/temp-00018669
>       at 
> org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:191)
>  ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       at 
> org.apache.flink.runtime.blob.AbstractBlobCache.getFileInternal(AbstractBlobCache.java:181)
>  ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       at 
> org.apache.flink.runtime.blob.PermanentBlobCache.getFile(PermanentBlobCache.java:206)
>  ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       at 
> org.apache.flink.runtime.execution.librarycache.BlobLibraryCacheManager.registerTask(BlobLibraryCacheManager.java:120)
>  ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       at 
> org.apache.flink.runtime.taskmanager.Task.createUserCodeClassloader(Task.java:863)
>  [flink-dist_2.11-1.5.0.jar:1.5.0]
>       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:579) 
> [flink-dist_2.11-1.5.0.jar:1.5.0]
>       at java.lang.Thread.run(Thread.java:748) [?:1.8.0_171]
> Caused by: java.io.IOException: Could not connect to BlobServer at address 
> flink-jobmanager:6124
>       at org.apache.flink.runtime.blob.BlobClient.<init>(BlobClient.java:124) 
> ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       at 
> org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:165)
>  ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       ... 6 more
> Caused by: java.net.UnknownHostException: jobmanager
>       at 
> java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:184) 
> ~[?:1.8.0_171]
>       at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392) 
> ~[?:1.8.0_171]
>       at java.net.Socket.connect(Socket.java:589) ~[?:1.8.0_171]
>       at java.net.Socket.connect(Socket.java:538) ~[?:1.8.0_171]
>       at org.apache.flink.runtime.blob.BlobClient.<init>(BlobClient.java:118) 
> ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       at 
> org.apache.flink.runtime.blob.BlobClient.downloadFromBlobServer(BlobClient.java:165)
>  ~[flink-dist_2.11-1.5.0.jar:1.5.0]
>       ... 6 more
> {noformat}
> Since the {{InetSocketAddress}} is re-used, you'll have repeated failures of 
> any tasks that are executed on that task manager and the only current 
> workaround is to manually restart the task manager.



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

Reply via email to