KarmaGYZ commented on code in PR #22137:
URL: https://github.com/apache/flink/pull/22137#discussion_r1227428286


##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java:
##########
@@ -116,12 +85,11 @@ public class AbstractTaskManagerFileHandlerTest extends 
TestLogger {
 
     private TransientBlobKey transientBlobKey2;
 
-    @BeforeClass
+    @BeforeAll
     public static void setup() throws IOException, HandlerRequestException {

Review Comment:
   Why we need to keep the public modifier? Same as below.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandler.java:
##########
@@ -151,34 +151,12 @@ protected CompletableFuture<Void> respondToRequest(
                         },
                         ctx.executor());
 
-        return resultFuture.whenComplete(
+        return resultFuture.handle(
                 (Void ignored, Throwable throwable) -> {
                     if (throwable != null) {
-                        log.error(
-                                "Failed to transfer file from TaskExecutor 
{}.",
-                                taskManagerId,
-                                throwable);
-                        fileBlobKeys.invalidate(taskManagerId);
-
-                        final Throwable strippedThrowable =
-                                
ExceptionUtils.stripCompletionException(throwable);
-
-                        if (strippedThrowable instanceof 
UnknownTaskExecutorException) {
-                            throw new CompletionException(
-                                    new NotFoundException(
-                                            String.format(
-                                                    "Failed to transfer file 
from TaskExecutor %s because it was unknown.",
-                                                    taskManagerId),
-                                            strippedThrowable));
-                        } else {
-                            throw new CompletionException(
-                                    new FlinkException(
-                                            String.format(
-                                                    "Failed to transfer file 
from TaskExecutor %s.",
-                                                    taskManagerId),
-                                            strippedThrowable));
-                        }
+                        handleException(ctx, httpRequest, throwable, 
taskManagerId);
                     }
+                    return ignored;

Review Comment:
   Not sure why we need to return here?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TestingChannelHandlerContext.java:
##########
@@ -46,11 +48,21 @@ class TestingChannelHandlerContext implements 
ChannelHandlerContext {
 
     final File outputFile;
     final ChannelPipeline pipeline = new TestingChannelPipeline();
+    HttpResponse httpResponse;
+    byte[] responseData;

Review Comment:
   Ditto. private modifier.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java:
##########
@@ -165,6 +168,81 @@ void testFileCacheExpiration() throws Exception {
         assertThat(FileUtils.readFileUtf8(outputFile)).isEqualTo(fileContent2);
     }
 
+    @Test
+    void testStdoutFileHandlerHandleFileNotFoundException() throws Exception {
+        final Time cacheEntryDuration = Time.milliseconds(1000L);
+        TestingTaskManagerStdoutFileHandler 
testingTaskManagerStdoutFileHandler =
+                createTestTaskManagerStdoutFileHandler(
+                        cacheEntryDuration, new FileNotFoundException("file 
not found"));
+        final File outputFile = TempDirUtils.newFile(temporaryFolder.toPath());
+        final TestingChannelHandlerContext testingContext =
+                new TestingChannelHandlerContext(outputFile);
+
+        CompletableFuture<Void> handleFuture =
+                testingTaskManagerStdoutFileHandler.respondToRequest(
+                        testingContext, HTTP_REQUEST, handlerRequest, null);
+        assertThat(handleFuture).isCompleted();
+        assertThat(testingContext.getHttpResponse())
+                .isNotNull()
+                .satisfies(
+                        httpResponse ->
+                                
assertThat(httpResponse.status()).isEqualTo(HttpResponseStatus.OK));
+        assertThat(testingContext.getResponseData())
+                .isNotNull()
+                .satisfies(
+                        data ->
+                                assertThat(new String(data, "UTF-8"))
+                                        .isEqualTo(
+                                                
TaskManagerStdoutFileHandler.FILE_NOT_FOUND_INFO));
+    }
+
+    @Test
+    void testStdoutFileHandlerHandleOtherException() throws Exception {
+        final Time cacheEntryDuration = Time.milliseconds(1000L);
+        TestingTaskManagerStdoutFileHandler 
testingTaskManagerStdoutFileHandler =
+                createTestTaskManagerStdoutFileHandler(
+                        cacheEntryDuration, new FlinkException("excepted 
exception"));
+        testHandleExceptionThrows(testingTaskManagerStdoutFileHandler);
+    }

Review Comment:
   We might file a new `TaskManagerStdoutFileHandlerTest` for it.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/taskmanager/TaskManagerStdoutFileHandler.java:
##########
@@ -24,22 +24,33 @@
 import org.apache.flink.runtime.blob.TransientBlobService;
 import org.apache.flink.runtime.clusterframework.types.ResourceID;
 import org.apache.flink.runtime.resourcemanager.ResourceManagerGateway;
+import org.apache.flink.runtime.rest.handler.util.HandlerUtils;
 import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
 import org.apache.flink.runtime.rest.messages.UntypedResponseMessageHeaders;
 import 
org.apache.flink.runtime.rest.messages.taskmanager.TaskManagerMessageParameters;
 import org.apache.flink.runtime.taskexecutor.FileType;
 import org.apache.flink.runtime.taskexecutor.TaskExecutor;
 import org.apache.flink.runtime.webmonitor.RestfulGateway;
 import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+import org.apache.flink.util.ExceptionUtils;
+
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpRequest;
+import 
org.apache.flink.shaded.netty4.io.netty.handler.codec.http.HttpResponseStatus;
 
 import javax.annotation.Nonnull;
 
+import java.io.FileNotFoundException;
+import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.CompletableFuture;
 
 /** Rest handler which serves the stdout file of the {@link TaskExecutor}. */
 public class TaskManagerStdoutFileHandler
         extends AbstractTaskManagerFileHandler<TaskManagerMessageParameters> {
+    static final String FILE_NOT_FOUND_INFO =

Review Comment:
   Add `@VisibleForTesting` here.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TestingChannelHandlerContext.java:
##########
@@ -0,0 +1,277 @@
+/*
+ * 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.taskmanager;
+
+import org.apache.flink.util.Preconditions;
+
+import org.apache.flink.shaded.netty4.io.netty.buffer.ByteBufAllocator;
+import org.apache.flink.shaded.netty4.io.netty.channel.Channel;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelFuture;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandler;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelHandlerContext;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPipeline;
+import 
org.apache.flink.shaded.netty4.io.netty.channel.ChannelProgressivePromise;
+import org.apache.flink.shaded.netty4.io.netty.channel.ChannelPromise;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultChannelPromise;
+import org.apache.flink.shaded.netty4.io.netty.channel.DefaultFileRegion;
+import 
org.apache.flink.shaded.netty4.io.netty.channel.embedded.EmbeddedChannel;
+import org.apache.flink.shaded.netty4.io.netty.util.Attribute;
+import org.apache.flink.shaded.netty4.io.netty.util.AttributeKey;
+import org.apache.flink.shaded.netty4.io.netty.util.concurrent.EventExecutor;
+import 
org.apache.flink.shaded.netty4.io.netty.util.concurrent.ImmediateEventExecutor;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.net.SocketAddress;
+
+/** Testing implementation of {@link ChannelHandlerContext}. */
+class TestingChannelHandlerContext implements ChannelHandlerContext {
+
+    final File outputFile;
+    final ChannelPipeline pipeline = new TestingChannelPipeline();

Review Comment:
   Why not add private modifier here?



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/TestingTaskManagerFileHandler.java:
##########
@@ -49,6 +48,8 @@
 
     private final ResourceID expectedTaskManagerId;
 
+    final Exception exceptionForRequestFileUpload;

Review Comment:
   private modifier.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/taskmanager/AbstractTaskManagerFileHandlerTest.java:
##########
@@ -165,6 +168,81 @@ void testFileCacheExpiration() throws Exception {
         assertThat(FileUtils.readFileUtf8(outputFile)).isEqualTo(fileContent2);
     }
 
+    @Test
+    void testStdoutFileHandlerHandleFileNotFoundException() throws Exception {
+        final Time cacheEntryDuration = Time.milliseconds(1000L);
+        TestingTaskManagerStdoutFileHandler 
testingTaskManagerStdoutFileHandler =
+                createTestTaskManagerStdoutFileHandler(
+                        cacheEntryDuration, new FileNotFoundException("file 
not found"));
+        final File outputFile = TempDirUtils.newFile(temporaryFolder.toPath());
+        final TestingChannelHandlerContext testingContext =
+                new TestingChannelHandlerContext(outputFile);
+
+        CompletableFuture<Void> handleFuture =
+                testingTaskManagerStdoutFileHandler.respondToRequest(
+                        testingContext, HTTP_REQUEST, handlerRequest, null);
+        assertThat(handleFuture).isCompleted();
+        assertThat(testingContext.getHttpResponse())
+                .isNotNull()
+                .satisfies(
+                        httpResponse ->
+                                
assertThat(httpResponse.status()).isEqualTo(HttpResponseStatus.OK));
+        assertThat(testingContext.getResponseData())
+                .isNotNull()
+                .satisfies(
+                        data ->
+                                assertThat(new String(data, "UTF-8"))
+                                        .isEqualTo(
+                                                
TaskManagerStdoutFileHandler.FILE_NOT_FOUND_INFO));
+    }
+
+    @Test
+    void testStdoutFileHandlerHandleOtherException() throws Exception {
+        final Time cacheEntryDuration = Time.milliseconds(1000L);
+        TestingTaskManagerStdoutFileHandler 
testingTaskManagerStdoutFileHandler =
+                createTestTaskManagerStdoutFileHandler(
+                        cacheEntryDuration, new FlinkException("excepted 
exception"));
+        testHandleExceptionThrows(testingTaskManagerStdoutFileHandler);
+    }
+
+    @Test
+    void testDefaultHandlerHandleFileNotFoundException() throws Exception {
+        final Time cacheEntryDuration = Time.milliseconds(1000L);
+        final Queue<CompletableFuture<TransientBlobKey>> requestFileUploads = 
new ArrayDeque<>(1);
+        TestingTaskManagerFileHandler testTaskManagerFileHandler =
+                createTestTaskManagerFileHandler(
+                        cacheEntryDuration,
+                        requestFileUploads,
+                        EXPECTED_TASK_MANAGER_ID,
+                        new FileNotFoundException());
+        testHandleExceptionThrows(testTaskManagerFileHandler);
+    }
+
+    @Test
+    void testDefaultHandlerHandleOtherException() throws Exception {
+        final Time cacheEntryDuration = Time.milliseconds(1000L);
+        final Queue<CompletableFuture<TransientBlobKey>> requestFileUploads = 
new ArrayDeque<>(1);
+        TestingTaskManagerFileHandler testTaskManagerFileHandler =
+                createTestTaskManagerFileHandler(
+                        cacheEntryDuration, requestFileUploads, 
EXPECTED_TASK_MANAGER_ID);
+        testHandleExceptionThrows(testTaskManagerFileHandler);
+    }

Review Comment:
   Not sure whether these  two tests are necessary. We don't change the 
behavior of `AbstractTaskManagerFileHandler`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to