zentol commented on code in PR #22882:
URL: https://github.com/apache/flink/pull/22882#discussion_r1246482509
##########
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/coordination/ClientCoordinationHandler.java:
##########
@@ -77,21 +77,29 @@ protected CompletableFuture<ClientCoordinationResponseBody>
handleRequest(
OperatorID operatorId =
request.getPathParameter(OperatorIDPathParameter.class);
SerializedValue<CoordinationRequest> serializedRequest =
request.getRequestBody().getSerializedCoordinationRequest();
- CompletableFuture<CoordinationResponse> responseFuture =
- gateway.deliverCoordinationRequestToCoordinator(
- jobId, operatorId, serializedRequest, timeout);
- return responseFuture.thenApply(
- coordinationResponse -> {
- try {
- return new ClientCoordinationResponseBody(
- new SerializedValue<>(coordinationResponse));
- } catch (IOException e) {
- throw new CompletionException(
- new RestHandlerException(
- "Failed to serialize coordination
response",
-
HttpResponseStatus.INTERNAL_SERVER_ERROR,
- e));
- }
- });
+ try {
+ CompletableFuture<CoordinationResponse> responseFuture =
+ gateway.deliverCoordinationRequestToCoordinator(
+ jobId, operatorId, serializedRequest, timeout);
+ return responseFuture.thenApply(
+ coordinationResponse -> {
+ try {
+ return new ClientCoordinationResponseBody(
+ new
SerializedValue<>(coordinationResponse));
+ } catch (IOException e) {
+ throw new CompletionException(
+ new RestHandlerException(
+ "Failed to serialize coordination
response",
+
HttpResponseStatus.INTERNAL_SERVER_ERROR,
+ e));
+ }
+ });
+ } catch (Exception e) {
+ throw new RestHandlerException(
Review Comment:
Who is throwing an exception here? Errors shoud be returned via the returned
future; they should never fail immediately.
##########
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/AbstractHandler.java:
##########
@@ -268,12 +266,7 @@ private CompletableFuture<Void> handleException(
HttpResponseStatus.SERVICE_UNAVAILABLE,
responseHeaders);
} else {
- if (throwable instanceof UnavailableDispatcherOperationException
- || throwable instanceof FileNotFoundException) {
- log.debug("Job is not initialized or is finished: {}",
throwable.getMessage());
- } else {
- log.error("Unhandled exception.", throwable);
- }
+ log.error("Unhandled exception.", throwable);
Review Comment:
I don't understand this change.
If you want an e2e test to ignore a certain exceptions in the logs, then
ignore that exception in `common.sh`.
##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/coordination/ClientCoordinationHandlerTest.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.job.coordination;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import
org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationHeaders;
+import
org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationMessageParameters;
+import
org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationRequestBody;
+import org.apache.flink.runtime.webmonitor.TestingRestfulGateway;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for {@link ClientCoordinationHandler}. */
+public class ClientCoordinationHandlerTest {
+ private static final JobID TEST_JOB_ID = new JobID();
+ private ClientCoordinationHandler clientCoordinationHandler;
+ private HandlerRequest<ClientCoordinationRequestBody> testRequest;
+
+ @BeforeEach
+ public void setUp() throws Exception {
+ final TestingRestfulGateway testingRestfulGateway =
+ new TestingRestfulGateway.Builder().build();
+
+ clientCoordinationHandler =
+ new ClientCoordinationHandler(
+ () ->
CompletableFuture.completedFuture(testingRestfulGateway),
+ Time.seconds(10),
+ Collections.emptyMap(),
+ ClientCoordinationHeaders.getInstance());
+
+ Map<String, String> pathParameters = new HashMap<>();
+ pathParameters.put("jobid", TEST_JOB_ID.toString());
+ pathParameters.put(
+ "operatorid", OperatorID.fromJobVertexID(new
JobVertexID()).toHexString());
+ testRequest =
+ HandlerRequest.resolveParametersAndCreate(
+ new ClientCoordinationRequestBody(
+ new SerializedValue<>(new
TestClientCoordinationRequest())),
+ new ClientCoordinationMessageParameters(),
+ pathParameters,
+ Collections.emptyMap(),
+ Collections.emptyList());
+ }
+
+ @Test
+ public void testGetResponseException() {
+ final TestingRestfulGateway testingRestfulGateway =
+ new TestingRestfulGateway.Builder()
+ .setDeliverCoordinationRequestToCoordinatorFunction(
+ (jobId, operatorId, request) -> {
+ throw new RuntimeException("Throw runtime
exception");
+ })
Review Comment:
return a failed future instead; you're testing behavior that doesnt occur at
runtime.
##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/coordination/ClientCoordinationHandlerTest.java:
##########
@@ -0,0 +1,93 @@
+/*
+ * 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.job.coordination;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.time.Time;
+import org.apache.flink.runtime.jobgraph.JobVertexID;
+import org.apache.flink.runtime.jobgraph.OperatorID;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import
org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationHeaders;
+import
org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationMessageParameters;
+import
org.apache.flink.runtime.rest.messages.job.coordination.ClientCoordinationRequestBody;
+import org.apache.flink.runtime.webmonitor.TestingRestfulGateway;
+import org.apache.flink.util.SerializedValue;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+/** Tests for {@link ClientCoordinationHandler}. */
+public class ClientCoordinationHandlerTest {
+ private static final JobID TEST_JOB_ID = new JobID();
+ private ClientCoordinationHandler clientCoordinationHandler;
+ private HandlerRequest<ClientCoordinationRequestBody> testRequest;
+
+ @BeforeEach
+ public void setUp() throws Exception {
+ final TestingRestfulGateway testingRestfulGateway =
+ new TestingRestfulGateway.Builder().build();
+
+ clientCoordinationHandler =
+ new ClientCoordinationHandler(
+ () ->
CompletableFuture.completedFuture(testingRestfulGateway),
+ Time.seconds(10),
+ Collections.emptyMap(),
+ ClientCoordinationHeaders.getInstance());
+
+ Map<String, String> pathParameters = new HashMap<>();
+ pathParameters.put("jobid", TEST_JOB_ID.toString());
+ pathParameters.put(
+ "operatorid", OperatorID.fromJobVertexID(new
JobVertexID()).toHexString());
+ testRequest =
+ HandlerRequest.resolveParametersAndCreate(
+ new ClientCoordinationRequestBody(
+ new SerializedValue<>(new
TestClientCoordinationRequest())),
+ new ClientCoordinationMessageParameters(),
+ pathParameters,
+ Collections.emptyMap(),
+ Collections.emptyList());
+ }
+
+ @Test
+ public void testGetResponseException() {
+ final TestingRestfulGateway testingRestfulGateway =
+ new TestingRestfulGateway.Builder()
+ .setDeliverCoordinationRequestToCoordinatorFunction(
+ (jobId, operatorId, request) -> {
+ throw new RuntimeException("Throw runtime
exception");
+ })
+ .build();
+ assertFalse(
+ assertThrows(
+ RestHandlerException.class,
+ () ->
+
clientCoordinationHandler.handleRequest(
+ testRequest,
testingRestfulGateway))
+ .logException());
Review Comment:
Use `FlinkAssertions#assertThatFuture`
--
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]