This is an automated email from the ASF dual-hosted git repository. zhuzh pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/flink.git
commit c7b1f4be424396bb07cc65777fdb31053f785be1 Author: JunRuiLee <[email protected]> AuthorDate: Fri Aug 30 13:41:41 2024 +0800 [FLINK-36185][core] Remove RuntimeContext#getExecutionConfig This closes #25270. --- .../apache/flink/api/common/functions/RuntimeContext.java | 12 ------------ .../api/common/functions/util/AbstractRuntimeUDFContext.java | 6 ------ .../org/apache/flink/cep/operator/CepRuntimeContext.java | 7 ------- .../flink/state/api/runtime/SavepointRuntimeContext.java | 7 ------- .../flink/streaming/util/MockStreamingRuntimeContext.java | 7 +++++++ .../streaming/api/functions/async/RichAsyncFunction.java | 7 ------- pom.xml | 4 ++++ 7 files changed, 11 insertions(+), 39 deletions(-) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java index e783d293864..66970e2440d 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java @@ -20,7 +20,6 @@ package org.apache.flink.api.common.functions; import org.apache.flink.annotation.Public; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobInfo; import org.apache.flink.api.common.TaskInfo; @@ -179,17 +178,6 @@ public interface RuntimeContext { return getTaskInfo().getTaskNameWithSubtasks(); } - /** - * Returns the {@link org.apache.flink.api.common.ExecutionConfig} for the currently executing - * job. - * - * @deprecated This method has been deprecated and will be removed in the upcoming FLINK major - * version FLINK-2.0. Users relying on this method should migrate to alternative getter - * methods, such as {@link #getGlobalJobParameters()} or {@link #isObjectReuseEnabled()}. - */ - @Deprecated - ExecutionConfig getExecutionConfig(); - /** * Create a serializer for a given type. * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java index a5e36599b89..75efc2a82d8 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/AbstractRuntimeUDFContext.java @@ -90,12 +90,6 @@ public abstract class AbstractRuntimeUDFContext implements RuntimeContext { this.metrics = metrics; } - @Override - @Deprecated - public ExecutionConfig getExecutionConfig() { - return executionConfig; - } - @Override public <T> TypeSerializer<T> createSerializer(TypeInformation<T> typeInformation) { return typeInformation.createSerializer(executionConfig.getSerializerConfig()); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepRuntimeContext.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepRuntimeContext.java index fb33f9c5eb6..ee8f7dd77dd 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepRuntimeContext.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/operator/CepRuntimeContext.java @@ -19,7 +19,6 @@ package org.apache.flink.cep.operator; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobInfo; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; @@ -73,12 +72,6 @@ class CepRuntimeContext implements RuntimeContext { return runtimeContext.getMetricGroup(); } - @Override - @Deprecated - public ExecutionConfig getExecutionConfig() { - return runtimeContext.getExecutionConfig(); - } - @Override public <T> TypeSerializer<T> createSerializer(TypeInformation<T> typeInformation) { return runtimeContext.createSerializer(typeInformation); diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointRuntimeContext.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointRuntimeContext.java index 8190c9b0c13..e39d18c68a6 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointRuntimeContext.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/runtime/SavepointRuntimeContext.java @@ -19,7 +19,6 @@ package org.apache.flink.state.api.runtime; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobInfo; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; @@ -98,12 +97,6 @@ public final class SavepointRuntimeContext implements RuntimeContext { return ctx.getMetricGroup(); } - @Override - @Deprecated - public ExecutionConfig getExecutionConfig() { - return ctx.getExecutionConfig(); - } - @Override public <T> TypeSerializer<T> createSerializer(TypeInformation<T> typeInformation) { return ctx.createSerializer(typeInformation); diff --git a/flink-runtime/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java b/flink-runtime/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java index 6676a3b4526..887ed968171 100644 --- a/flink-runtime/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java +++ b/flink-runtime/src/test/java/org/apache/flink/streaming/util/MockStreamingRuntimeContext.java @@ -39,6 +39,8 @@ public class MockStreamingRuntimeContext extends StreamingRuntimeContext { private final int numParallelSubtasks; private final int subtaskIndex; + private final MockEnvironment environment; + public MockStreamingRuntimeContext( boolean isCheckpointingEnabled, int numParallelSubtasks, int subtaskIndex) { @@ -66,6 +68,11 @@ public class MockStreamingRuntimeContext extends StreamingRuntimeContext { this.isCheckpointingEnabled = isCheckpointingEnabled; this.numParallelSubtasks = numParallelSubtasks; this.subtaskIndex = subtaskIndex; + this.environment = environment; + } + + public ExecutionConfig getExecutionConfig() { + return environment.getExecutionConfig(); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java index 2a727c69dcf..5f9b33fe7e5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/async/RichAsyncFunction.java @@ -19,7 +19,6 @@ package org.apache.flink.streaming.api.functions.async; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobInfo; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; @@ -114,12 +113,6 @@ public abstract class RichAsyncFunction<IN, OUT> extends AbstractRichFunction return runtimeContext.getMetricGroup(); } - @Override - @Deprecated - public ExecutionConfig getExecutionConfig() { - return runtimeContext.getExecutionConfig(); - } - @Override public <T> TypeSerializer<T> createSerializer(TypeInformation<T> typeInformation) { return runtimeContext.createSerializer(typeInformation); diff --git a/pom.xml b/pom.xml index 3c2aabc8478..777fea41f31 100644 --- a/pom.xml +++ b/pom.xml @@ -2376,6 +2376,10 @@ under the License. <exclude>org.apache.flink.streaming.api.functions.source.RichSourceFunction</exclude> <exclude>org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction</exclude> <exclude>org.apache.flink.streaming.api.functions.source.ParallelSourceFunction</exclude> + <!-- FLINK-33712 deprecated the RuntimeContext#getExecutionConfig; this will be removed in Flink 2.0. --> + <exclude> + org.apache.flink.api.common.functions.RuntimeContext#getExecutionConfig() + </exclude> <!-- MARKER: end exclusions --> </excludes> <accessModifier>public</accessModifier>
