Skip to content

[FLINK-39316][runtime] Fix BlobServer address resolution failing with VPN networking#27820

Open
ddebowczyk92 wants to merge 1 commit intoapache:masterfrom
ddebowczyk92:FLINK-39316
Open

[FLINK-39316][runtime] Fix BlobServer address resolution failing with VPN networking#27820
ddebowczyk92 wants to merge 1 commit intoapache:masterfrom
ddebowczyk92:FLINK-39316

Conversation

@ddebowczyk92
Copy link
Contributor

… VPN networking

What is the purpose of the change

BlobServer.getAddress() falls back to InetAddress.getLocalHost() when bound to the wildcard address (0.0.0.0). On machines where the hostname resolves to a non-local IP (e.g. VPN), this returns an unreachable address, causing blob uploads to fail with Connection reset during job submission. This was introduced by FLINK-38109.

Brief change log

  • Changed BlobServer.getAddress() to use InetAddress.getLoopbackAddress() instead of InetAddress.getLocalHost() when bound to the wildcard address

Verifying this change

Please make sure both new and modified tests in this PR follow the conventions for tests defined in our code quality guide.

This change is already covered by existing tests, such as FunctionITCase#testUsingAddJar.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no)
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (no)

@flinkbot
Copy link
Collaborator

flinkbot commented Mar 24, 2026

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run azure re-run the last Azure build

@gaborgsomogyi
Copy link
Contributor

config.set(JobManagerOptions.BIND_HOST, loopbackAddress); is not working?

@ddebowczyk92
Copy link
Contributor Author

ddebowczyk92 commented Mar 25, 2026

@gaborgsomogyi Hey, thanks for your input. The thing with that suggestion is that I'm not sure where I would apply this configuration. BlobServer starts immediately after MiniClusterExtension is spun up, so if I tried to apply this config in StreamingTestBase.before like below:

  @throws(classOf[Exception])
  @BeforeEach
  def before(): Unit = {
    this.env = StreamExecutionEnvironment.getExecutionEnvironment
    env.setParallelism(4)
    if (enableObjectReuse) {
      this.env.getConfig.enableObjectReuse()
    }
    val setting = EnvironmentSettings.newInstance().inStreamingMode().build()
    this.tEnv = StreamTableEnvironment.create(env, setting)
    this.tEnv.getConfig.set(JobManagerOptions.BIND_HOST, "127.0.0.1")
  }

it would definitely be too late. Anyway, I think we're drifting a bit from the purpose of this change. As I described in more detail in the JIRA ticket FLINK-39316, the problem I'm trying to solve occurs under certain circumstances. When I was trying to run FunctionITCase in my IDE, multiple test cases turned red with the same error:

Caused by: java.io.IOException: PUT operation failed: Connection reset
	at org.apache.flink.runtime.blob.BlobClient.putInputStream(BlobClient.java:496)
	at org.apache.flink.runtime.blob.BlobClient.uploadFile(BlobClient.java:545)
	at org.apache.flink.runtime.client.ClientUtils.uploadUserJars(ClientUtils.java:115)
	at org.apache.flink.runtime.client.ClientUtils.uploadAndSetUserJars(ClientUtils.java:107)
	at org.apache.flink.runtime.client.ClientUtils.uploadExecutionPlanFiles(ClientUtils.java:85)
	... 80 more
Caused by: java.net.SocketException: Connection reset
	at java.base/sun.nio.ch.NioSocketImpl.implRead(NioSocketImpl.java:328)
	at java.base/sun.nio.ch.NioSocketImpl.read(NioSocketImpl.java:355)
	at java.base/sun.nio.ch.NioSocketImpl$1.read(NioSocketImpl.java:808)
	at java.base/java.net.Socket$SocketInputStream.read(Socket.java:966)
	at java.base/java.net.Socket$SocketInputStream.read(Socket.java:961)
	at org.apache.flink.runtime.blob.BlobOutputStream.receiveAndCheckPutResponse(BlobOutputStream.java:175)
	at org.apache.flink.runtime.blob.BlobOutputStream.finish(BlobOutputStream.java:119)
	at org.apache.flink.runtime.blob.BlobClient.putInputStream(BlobClient.java:493)

It took me a non-trivial amount of time to figure out what was actually going on and how to prevent it. I believe this small change would prevent many headaches for developers who work behind corporate VPNs

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants