Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[FLINK-35227][rest] Remove execution-mode in ExecutionConfigInfo #25080

Merged
merged 2 commits into from
Jul 18, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 0 additions & 2 deletions docs/static/generated/rest_v1_dispatcher.yml
Original file line number Diff line number Diff line change
Expand Up @@ -2079,8 +2079,6 @@ components:
ExecutionConfigInfo:
type: object
properties:
execution-mode:
type: string
job-parallelism:
type: integer
format: int32
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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\":{}}}");
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;

Expand Down Expand Up @@ -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;

Expand All @@ -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<String, String> 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;
}
Expand Down Expand Up @@ -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(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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);
}
}