[ 
https://issues.apache.org/jira/browse/FLINK-39316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated FLINK-39316:
-----------------------------------
    Labels: pull-request-available  (was: )

> BlobServer.getAddress() uses InetAddress.getLocalHost() which fails with VPN 
> networking
> ---------------------------------------------------------------------------------------
>
>                 Key: FLINK-39316
>                 URL: https://issues.apache.org/jira/browse/FLINK-39316
>             Project: Flink
>          Issue Type: Bug
>          Components: Runtime / Coordination
>    Affects Versions: 2.1.0, 2.2.0, 2.1.1
>            Reporter: Dominik Dębowczyk
>            Priority: Major
>              Labels: pull-request-available
>
> {{BlobServer.getAddress()}} returns an unreachable address when the server is 
> bound to the wildcard address (0.0.0.0) and the machine's hostname resolves 
> to a non-local IP (e.g. a VPN address). This causes blob uploads to fail with 
> Connection reset during job submission.
> h3. Root Cause
> FLINK-38109 changed {{MiniCluster.createBlobServerAddress()}} from using the 
> Dispatcher's RPC hostname (typically localhost) to using 
> {{{}BlobServer.getAddress(){}}}. When the BlobServer binds to 0.0.0.0, 
> {{getAddress()}} falls back to {{InetAddress.getLocalHost()}}
> {{InetAddress.getLocalHost()}} resolves the machine's hostname via DNS. On 
> machines with VPN software (e.g. corporate VPNs), the hostname can resolve to 
> a VPN-assigned IP that is not directly reachable on any local interface.
> The TCP connection completes at the kernel level (routed through the VPN) but 
> the packets never reach the local BlobServer's accept queue. The BlobServer 
> never processes the request, and the client gets a Connection reset when 
> reading the response.
> h3. How to Reproduce
> Run any test that submits a job with user jars through the {{MiniCluster}} on 
> a machine where the hostname resolves to a non-local-interface IP:
> {{mvn test -pl flink-table/flink-table-planner 
> -Dtest="FunctionITCase#testUsingAddJar"}}
> Fails with:
> {code:java}
> Caused by: java.io.IOException: PUT operation failed: Connection reset
>     at o.a.f.runtime.blob.BlobClient.putInputStream(BlobClient.java:496)
>     at o.a.f.runtime.blob.BlobClient.uploadFile(BlobClient.java:545)
> Caused by: java.net.SocketException: Connection reset
>     at 
> o.a.f.runtime.blob.BlobOutputStream.receiveAndCheckPutResponse(BlobOutputStream.java:175){code}
> h3. Fix
> Replace {{InetAddress.getLocalHost()}} with 
> {{InetAddress.getLoopbackAddress()}} in {{{}BlobServer.getAddress(){}}}. When 
> a server binds to 0.0.0.0 (all interfaces), the loopback address (127.0.0.1) 
> is always a valid way to reach it locally. This avoids the dependency on DNS 
> hostname resolution which is unreliable across different network 
> configurations.
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to