godfreyhe commented on a change in pull request #12037:
URL: https://github.com/apache/flink/pull/12037#discussion_r422751031



##########
File path: 
flink-runtime/src/test/java/org/apache/flink/runtime/operators/coordination/OperatorCoordinatorSchedulerTest.java
##########
@@ -163,6 +163,36 @@ public void taskTaskManagerFailuresAreReportedBack() 
throws Exception {
                assertThat(result, futureFailedWith(TestException.class));
        }
 
+       @Test
+       @SuppressWarnings("unchecked")
+       public void testDeliveringClientRequestToResponser() throws Exception {
+               final OperatorCoordinator.Provider provider = new 
TestingCoordinationResponser.Provider(testOperatorId);
+               final DefaultScheduler scheduler = createScheduler(provider);
+
+               final String payload = "testing payload";
+               final TestingCoordinationResponser.Request<String> request =
+                       new TestingCoordinationResponser.Request<>(payload);
+               final TestingCoordinationResponser.Response<String> response =
+                       (TestingCoordinationResponser.Response<String>)
+                               
scheduler.deliverCoordinationRequestToCoordinator(testOperatorId, 
request).get();
+
+               assertEquals(payload, response.getPayload());
+       }
+
+       @Test
+       public void testDeliveringClientRequestToNonResponser() throws 
Exception {
+               final OperatorCoordinator.Provider provider = new 
TestingOperatorCoordinator.Provider(testOperatorId);
+               final DefaultScheduler scheduler = createScheduler(provider);
+
+               final String payload = "testing payload";
+               final TestingCoordinationResponser.Request<String> request =
+                       new TestingCoordinationResponser.Request<>(payload);
+               final CompletableFuture<CoordinationResponse> future =
+                       
scheduler.deliverCoordinationRequestToCoordinator(testOperatorId, request);
+
+               assertThat(future, 
futureFailedWith(IllegalArgumentException.class));
+       }
+

Review comment:
       add a test for scenario: "operator id not found"

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/SchedulerBase.java
##########
@@ -933,6 +941,22 @@ public void deliverOperatorEventToCoordinator(
                }
        }
 
+       @Override
+       public CompletableFuture<CoordinationResponse> 
deliverCoordinationRequestToCoordinator(
+                       OperatorID operator,
+                       CoordinationRequest request) throws FlinkException {
+               OperatorCoordinator coordinator = coordinatorMap.get(operator);
+               if (coordinator instanceof CoordinationResponser) {
+                       return CompletableFuture.completedFuture(
+                               ((CoordinationResponser) 
coordinator).handleCoordinationRequest(request));
+               } else if (coordinator != null) {
+                       return FutureUtils.completedExceptionally(
+                               new IllegalArgumentException("Coordinator of 
operator " + operator + " cannot handle client event"));

Review comment:
       I think we should throw `FlinkException`, this situation is the same as 
the operation does not exist

##########
File path: 
flink-clients/src/main/java/org/apache/flink/client/program/rest/RestClusterClient.java
##########
@@ -411,6 +418,36 @@ public void close() {
                return triggerSavepoint(jobId, savepointDirectory, false);
        }
 
+       @Override
+       public CompletableFuture<CoordinationResponse> sendCoordinationRequest(
+                       JobID jobId,
+                       OperatorID operatorId,
+                       CoordinationRequest request) {
+               ClientCoordinationHeaders headers = 
ClientCoordinationHeaders.getInstance();
+               ClientCoordinationMessageParameters params = new 
ClientCoordinationMessageParameters();
+               params.jobPathParameter.resolve(jobId);
+               params.operatorPathParameter.resolve(operatorId);
+
+               SerializedValue<CoordinationRequest> serializedRequest;
+               try {
+                       serializedRequest = new SerializedValue<>(request);
+               } catch (IOException e) {
+                       return FutureUtils.completedExceptionally(e);
+               }
+
+               ClientCoordinationRequestBody requestBody = new 
ClientCoordinationRequestBody(serializedRequest);
+               return sendRequest(headers, params, requestBody).thenApply(
+                       responseBody -> {
+                               try {
+                                       return responseBody
+                                               
.getSerializedCoordinationResponse()
+                                               
.deserializeValue(getClass().getClassLoader());
+                               } catch (IOException | ClassNotFoundException 
e) {
+                                       throw new RuntimeException("Failed to 
deserialize coordination response", e);

Review comment:
       throw `CompletionException` like `getAccumulators` method

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CoordinationResponser.java
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.operators.coordination;
+
+/**
+ * Coordinator interface which can handle {@link CoordinationRequest}s
+ * and response with {@link CoordinationResponse}s to the client.
+ */
+public interface CoordinationResponser {

Review comment:
       rename to `CoordinationHandler` ? similar to `OperatorEventHandler`

##########
File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/operators/coordination/CoordinationRequester.java
##########
@@ -0,0 +1,39 @@
+/*
+ * 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.operators.coordination;
+
+import org.apache.flink.runtime.jobgraph.OperatorID;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Client interface which sends out a {@link CoordinationRequest} and
+ * expects for a {@link CoordinationResponse} from a {@link 
OperatorCoordinator}.
+ */
+public interface CoordinationRequester {

Review comment:
       rename to `CoordinationRequestGateway`? similar to `OperatorEventGateway`




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Reply via email to