Rosa Seohwa Kang created FLINK-38109:
----------------------------------------

             Summary: BlobServer isn't associated with the correct bind-host
                 Key: FLINK-38109
                 URL: https://issues.apache.org/jira/browse/FLINK-38109
             Project: Flink
          Issue Type: Bug
          Components: Runtime / Coordination
    Affects Versions: 1.11.0
         Environment: !image-2025-07-16-18-23-30-302.png|width=654,height=135!
            Reporter: Rosa Seohwa Kang


In a flink cluster, if the JobManager bind-host is configured with an address, 
this creates a mismatch where clients expect to connect to BlobServer at same 
address, but it's actually bound to localhost. The BlobServer becomes 
unreachable to clients, causing "Could not connect to BlobServer" errors.

This can be reproduced when uploading a JAR file to the cluster for UDF 
registration, and configuring the JobManager bind-host to an external address.

Example:

```
package com.shopify.flink.examples.functions.asynchttpfunction;
 
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.RestOptions;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
 
public class RosaTestFunction {
    public static void main(final String[] args) throws Exception {
        
        // Configuration for connecting to the local session cluster
        final Configuration configuration = new Configuration();
 
        // Configure REST client to connect to the local session cluster
        configuration.set(RestOptions.ADDRESS, 
"flink-examples-flinkbook.root.shopify.dev.internal");
        configuration.set(RestOptions.PORT, 7100);  // REST API port
 
configuration.setString("jobmanager.bind-host", 
"flink-examples-flinkbook.root.shopify.dev.internal");
configuration.setString("taskmanager.bind-host", 
"flink-examples-flinkbook.root.shopify.dev.internal");
 
        StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment(configuration);
        StreamTableEnvironment tableEnv = StreamTableEnvironment.create(env);
 
        tableEnv.executeSql("""
                CREATE FUNCTION example_custom
                AS 'com.shopify.flink.examples.shared.ExampleCustomFunction'
                LANGUAGE JAVA
                USING JAR 
'file:///Users/rosa/src/github.com/Shopify/flink-examples/sql/pipelines/flinksql-example/jars/shared.jar';""");
 
        tableEnv.executeSql("SELECT example_custom(1,2) as 
sum_result;").print();
    }
}
```

Root cause:

The BlobServer bind-host is set to use the Jobmanager bind-host, however isn't 
assigned the correct host in the PekkoRpcService.

[BlobServer.java](https://github.com/apache/flink/blob/master/flink-runtime/src/main/java/org/apache/flink/runtime/blob/BlobServer.java#L218-L220)
```
String bindHost = 
(String)config.getOptional(JobManagerOptions.BIND_HOST).orElseGet(NetUtils::getWildcardIPAddress);
```

[PekkoRpcService.java](https://github.com/apache/flink/blob/master/flink-rpc/flink-rpc-akka/src/main/java/org/apache/flink/runtime/rpc/pekko/PekkoRpcService.java#L477-L486)

```
    private Tuple2<String, String> extractAddressHostname(ActorRef actorRef) {
        final String actorAddress = PekkoUtils.getRpcURL(actorSystem, actorRef);
        final String hostname;
        Option<String> host = actorRef.path().address().host();
        if (host.isEmpty()) {
            hostname = "localhost";
        } else {
            hostname = host.get();
        }

        return Tuple2.of(actorAddress, hostname);
    }
```

 

Proposed solution:

Add a `blob.server.bind-host` configuration option similar to the existing 
bind-host solution introduced in 
[FLINK-15911](https://issues.apache.org/jira/browse/FLINK-15911).

 



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

Reply via email to