This is an automated email from the ASF dual-hosted git repository.

ferenc-csaky pushed a commit to branch release-2.2
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-2.2 by this push:
     new 29d9a19dce5 [FLINK-39761][runtime] Set 'Connection: close' header in 
`StaticFileServerHandler` when sending '304 Not Modified' responses
29d9a19dce5 is described below

commit 29d9a19dce594973770048e8ebd222a9a10c0532
Author: nattilabalint <[email protected]>
AuthorDate: Fri Jun 5 15:57:52 2026 +0200

    [FLINK-39761][runtime] Set 'Connection: close' header in 
`StaticFileServerHandler` when sending '304 Not Modified' responses
---
 .../legacy/files/StaticFileServerHandler.java      |  4 ++
 .../legacy/files/StaticFileServerHandlerTest.java  | 64 ++++++++++++++++++++++
 2 files changed, 68 insertions(+)

diff --git 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
index 4454101d17c..71531b0a86d 100644
--- 
a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
+++ 
b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandler.java
@@ -45,6 +45,7 @@ import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultFullHtt
 import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.DefaultHttpResponse;
 import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
 import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpChunkedInput;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaderNames;
 import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaderValues;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
 import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponse;
@@ -308,6 +309,9 @@ public class StaticFileServerHandler<T extends 
RestfulGateway> extends LeaderRet
         FullHttpResponse response = new DefaultFullHttpResponse(HTTP_1_1, 
NOT_MODIFIED);
         setDateHeader(response);
 
+        // Explicitly notify the client that the connection will be dropped
+        response.headers().set(HttpHeaderNames.CONNECTION, 
HttpHeaderValues.CLOSE);
+
         // close the connection as soon as the error message is sent.
         ctx.writeAndFlush(response).addListener(ChannelFutureListener.CLOSE);
     }
diff --git 
a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandlerTest.java
 
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandlerTest.java
new file mode 100644
index 00000000000..ad7acaa549f
--- /dev/null
+++ 
b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/legacy/files/StaticFileServerHandlerTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.runtime.rest.handler.legacy.files;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import 
org.apache.flink.shaded.netty4.io.netty.channel.ChannelInboundHandlerAdapter;
+import 
org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.FullHttpResponse;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaderNames;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpHeaderValues;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+class StaticFileServerHandlerTest {
+
+    @Test
+    void testSendNotModifiedIncludesConnectionCloseHeader() {
+        EmbeddedChannel channel =
+                new EmbeddedChannel(
+                        new ChannelInboundHandlerAdapter() {
+                            @Override
+                            public void channelActive(ChannelHandlerContext 
ctx) {
+                                StaticFileServerHandler.sendNotModified(ctx);
+                            }
+                        });
+
+        FullHttpResponse response = channel.readOutbound();
+
+        assertThat(response).as("No response received!").isNotNull();
+
+        try {
+            
assertThat(response.status()).isEqualTo(HttpResponseStatus.NOT_MODIFIED);
+
+            assertThat(response.headers().contains(HttpHeaderNames.CONNECTION))
+                    .as("The 'Connection' header is missing!")
+                    .isTrue();
+
+            assertThat(response.headers().get(HttpHeaderNames.CONNECTION))
+                    .as("The value of 'Connection' header is not 'close'!")
+                    .isEqualTo(HttpHeaderValues.CLOSE.toString());
+        } finally {
+            response.release();
+        }
+    }
+}

Reply via email to