WeiZhong94 commented on code in PR #27580:
URL: https://github.com/apache/flink/pull/27580#discussion_r2867264175


##########
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescales/JobRescaleConfigHandler.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.rescales;
+
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.JobIDPathParameter;
+import org.apache.flink.runtime.rest.messages.JobMessageParameters;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import 
org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigHeaders;
+import 
org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigInfo;
+import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+/** The handler for job rescale configuration. */
+public class JobRescaleConfigHandler
+        extends AbstractExecutionGraphHandler<JobRescaleConfigInfo, 
JobMessageParameters>
+        implements JsonArchivist {
+
+    public JobRescaleConfigHandler(
+            GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+            Duration timeout,
+            Map<String, String> responseHeaders,
+            MessageHeaders<EmptyRequestBody, JobRescaleConfigInfo, 
JobMessageParameters>
+                    messageHeaders,
+            ExecutionGraphCache executionGraphCache,
+            Executor executor) {
+        super(
+                leaderRetriever,
+                timeout,
+                responseHeaders,
+                messageHeaders,
+                executionGraphCache,
+                executor);
+    }
+
+    @Override
+    protected JobRescaleConfigInfo handleRequest(
+            HandlerRequest<EmptyRequestBody> request, ExecutionGraphInfo 
executionGraphInfo)
+            throws RestHandlerException {
+        return executionGraphInfo.getJobRescaleConfigInfo();

Review Comment:
   If executionGraphInfo.getJobRescaleConfigInfo() returns null, throw 
exception will be a better choice. 



##########
flink-runtime/src/main/java/org/apache/flink/runtime/rest/handler/job/rescales/JobRescaleConfigHandler.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.rescales;
+
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler;
+import org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.JobIDPathParameter;
+import org.apache.flink.runtime.rest.messages.JobMessageParameters;
+import org.apache.flink.runtime.rest.messages.MessageHeaders;
+import 
org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigHeaders;
+import 
org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigInfo;
+import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
+import org.apache.flink.runtime.webmonitor.RestfulGateway;
+import org.apache.flink.runtime.webmonitor.history.ArchivedJson;
+import org.apache.flink.runtime.webmonitor.history.JsonArchivist;
+import org.apache.flink.runtime.webmonitor.retriever.GatewayRetriever;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.Executor;
+
+/** The handler for job rescale configuration. */
+public class JobRescaleConfigHandler
+        extends AbstractExecutionGraphHandler<JobRescaleConfigInfo, 
JobMessageParameters>
+        implements JsonArchivist {
+
+    public JobRescaleConfigHandler(
+            GatewayRetriever<? extends RestfulGateway> leaderRetriever,
+            Duration timeout,
+            Map<String, String> responseHeaders,
+            MessageHeaders<EmptyRequestBody, JobRescaleConfigInfo, 
JobMessageParameters>
+                    messageHeaders,
+            ExecutionGraphCache executionGraphCache,
+            Executor executor) {
+        super(
+                leaderRetriever,
+                timeout,
+                responseHeaders,
+                messageHeaders,
+                executionGraphCache,
+                executor);
+    }
+
+    @Override
+    protected JobRescaleConfigInfo handleRequest(
+            HandlerRequest<EmptyRequestBody> request, ExecutionGraphInfo 
executionGraphInfo)
+            throws RestHandlerException {
+        return executionGraphInfo.getJobRescaleConfigInfo();
+    }
+
+    @Override
+    public Collection<ArchivedJson> archiveJsonWithPath(ExecutionGraphInfo 
executionGraphInfo)
+            throws IOException {
+        return Collections.singletonList(
+                new ArchivedJson(
+                        JobRescaleConfigHeaders.getInstance()
+                                .getTargetRestEndpointURL()
+                                .replace(
+                                        ':' + JobIDPathParameter.KEY,
+                                        
executionGraphInfo.getJobId().toString()),
+                        executionGraphInfo.getJobRescaleConfigInfo()));

Review Comment:
   If executionGraphInfo.getJobRescaleConfigInfo() returns null, it will run 
into a NPE. we can return an ErrorResponseBody instead.



##########
flink-runtime/src/main/java/org/apache/flink/runtime/scheduler/ExecutionGraphInfo.java:
##########
@@ -43,6 +44,12 @@ public class ExecutionGraphInfo implements Serializable {
     private final Iterable<RootExceptionHistoryEntry> exceptionHistory;
     @Nullable private final JobManagerOptions.SchedulerType schedulerType;
 
+    /**
+     * The value is null when the job is not enabled {@link

Review Comment:
   How about:
   The value will be null if the job isn't using the {@link AdaptiveScheduler}.



##########
flink-runtime/src/test/java/org/apache/flink/runtime/rest/handler/job/rescales/JobRescaleConfigHandlerTest.java:
##########
@@ -0,0 +1,92 @@
+/*
+ * 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.rescales;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.configuration.SchedulerExecutionMode;
+import org.apache.flink.runtime.rest.handler.HandlerRequest;
+import org.apache.flink.runtime.rest.handler.HandlerRequestException;
+import org.apache.flink.runtime.rest.handler.RestHandlerException;
+import org.apache.flink.runtime.rest.handler.legacy.DefaultExecutionGraphCache;
+import 
org.apache.flink.runtime.rest.handler.legacy.utils.ArchivedExecutionGraphBuilder;
+import org.apache.flink.runtime.rest.messages.EmptyRequestBody;
+import org.apache.flink.runtime.rest.messages.JobIDPathParameter;
+import org.apache.flink.runtime.rest.messages.JobMessageParameters;
+import 
org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigHeaders;
+import 
org.apache.flink.runtime.rest.messages.job.rescales.JobRescaleConfigInfo;
+import org.apache.flink.runtime.scheduler.ExecutionGraphInfo;
+import org.apache.flink.testutils.TestingUtils;
+import org.apache.flink.util.concurrent.Executors;
+
+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.assertj.core.api.Assertions.assertThat;
+
+/*** Test for {@link JobRescaleConfigHandler}. */

Review Comment:
   `/***` -> `/**`



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