From cef48545b54986dc9690bbf6b0b9193d53e798ed Mon Sep 17 00:00:00 2001 From: Rui Fan <1996fanrui@gmail.com> Date: Thu, 18 Jul 2024 10:04:34 +0800 Subject: [PATCH] [FLINK-35227][rest] Remove execution-mode in ExecutionConfigInfo (#25080) --- docs/static/generated/rest_v1_dispatcher.yml | 2 -- .../runtime/webmonitor/WebFrontendITCase.java | 2 +- .../runtime/rest/messages/JobConfigInfo.java | 22 +------------------ .../rest/messages/JobConfigInfoTest.java | 3 +-- 4 files changed, 3 insertions(+), 26 deletions(-) diff --git a/docs/static/generated/rest_v1_dispatcher.yml b/docs/static/generated/rest_v1_dispatcher.yml index 0b060189e299b..1ea036fc53b4e 100644 --- a/docs/static/generated/rest_v1_dispatcher.yml +++ b/docs/static/generated/rest_v1_dispatcher.yml @@ -2079,8 +2079,6 @@ components: ExecutionConfigInfo: type: object properties: - execution-mode: - type: string job-parallelism: type: integer format: int32 diff --git a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java index 36171aff13ece..7da290ead1de7 100644 --- a/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java +++ b/flink-runtime-web/src/test/java/org/apache/flink/runtime/webmonitor/WebFrontendITCase.java @@ -355,7 +355,7 @@ void testCancel( "{\"jid\":\"" + jid + "\",\"name\":\"Stoppable streaming test job\"," - + "\"execution-config\":{\"execution-mode\":\"PIPELINED\",\"restart-strategy\":\"Cluster level default restart strategy\"," + + "\"execution-config\":{\"restart-strategy\":\"Cluster level default restart strategy\"," + "\"job-parallelism\":1,\"object-reuse-mode\":false,\"user-config\":{}}}"); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigInfo.java b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigInfo.java index ca7970d05a3b9..77d007cf2a25d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigInfo.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/rest/messages/JobConfigInfo.java @@ -22,7 +22,6 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.configuration.ConfigurationUtils; import org.apache.flink.runtime.rest.handler.job.JobConfigHandler; -import org.apache.flink.runtime.rest.messages.job.JobDetailsInfo; import org.apache.flink.runtime.rest.util.RestMapperUtils; import org.apache.flink.util.Preconditions; @@ -169,17 +168,11 @@ public JobConfigInfo deserialize( /** Nested class to encapsulate the execution configuration. */ public static final class ExecutionConfigInfo { - @Deprecated public static final String FIELD_NAME_EXECUTION_MODE = "execution-mode"; public static final String FIELD_NAME_RESTART_STRATEGY = "restart-strategy"; public static final String FIELD_NAME_PARALLELISM = "job-parallelism"; public static final String FIELD_NAME_OBJECT_REUSE_MODE = "object-reuse-mode"; public static final String FIELD_NAME_GLOBAL_JOB_PARAMETERS = "user-config"; - /** @deprecated Use {@link JobDetailsInfo#getJobType()} instead. */ - @Deprecated - @JsonProperty(FIELD_NAME_EXECUTION_MODE) - private final String executionMode; - @JsonProperty(FIELD_NAME_RESTART_STRATEGY) private final String restartStrategy; @@ -194,23 +187,17 @@ public static final class ExecutionConfigInfo { @JsonCreator public ExecutionConfigInfo( - @JsonProperty(FIELD_NAME_EXECUTION_MODE) String executionMode, @JsonProperty(FIELD_NAME_RESTART_STRATEGY) String restartStrategy, @JsonProperty(FIELD_NAME_PARALLELISM) int parallelism, @JsonProperty(FIELD_NAME_OBJECT_REUSE_MODE) boolean isObjectReuse, @JsonProperty(FIELD_NAME_GLOBAL_JOB_PARAMETERS) Map globalJobParameters) { - this.executionMode = Preconditions.checkNotNull(executionMode); this.restartStrategy = Preconditions.checkNotNull(restartStrategy); this.parallelism = parallelism; this.isObjectReuse = isObjectReuse; this.globalJobParameters = Preconditions.checkNotNull(globalJobParameters); } - public String getExecutionMode() { - return executionMode; - } - public String getRestartStrategy() { return restartStrategy; } @@ -239,24 +226,17 @@ public boolean equals(Object o) { ExecutionConfigInfo that = (ExecutionConfigInfo) o; return parallelism == that.parallelism && isObjectReuse == that.isObjectReuse - && Objects.equals(executionMode, that.executionMode) && Objects.equals(restartStrategy, that.restartStrategy) && Objects.equals(globalJobParameters, that.globalJobParameters); } @Override public int hashCode() { - return Objects.hash( - executionMode, - restartStrategy, - parallelism, - isObjectReuse, - globalJobParameters); + return Objects.hash(restartStrategy, parallelism, isObjectReuse, globalJobParameters); } public static ExecutionConfigInfo from(ArchivedExecutionConfig archivedExecutionConfig) { return new ExecutionConfigInfo( - archivedExecutionConfig.getExecutionMode(), archivedExecutionConfig.getRestartStrategyDescription(), archivedExecutionConfig.getParallelism(), archivedExecutionConfig.getObjectReuseEnabled(), diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobConfigInfoTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobConfigInfoTest.java index 5bd15aa7aaa53..7bef151a84fe6 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobConfigInfoTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/rest/messages/JobConfigInfoTest.java @@ -43,8 +43,7 @@ protected JobConfigInfo getTestResponseInstance() { globalJobParameters.put("hi", "ho"); final JobConfigInfo.ExecutionConfigInfo executionConfigInfo = - new JobConfigInfo.ExecutionConfigInfo( - "foobar", "always", 42, false, globalJobParameters); + new JobConfigInfo.ExecutionConfigInfo("always", 42, false, globalJobParameters); return new JobConfigInfo(new JobID(), "testJob", executionConfigInfo); } }