diff --git a/docs/content.zh/docs/dev/datastream/operators/asyncio.md b/docs/content.zh/docs/dev/datastream/operators/asyncio.md index 0caa5e8ce3b2e..3b0e3e4de0925 100644 --- a/docs/content.zh/docs/dev/datastream/operators/asyncio.md +++ b/docs/content.zh/docs/dev/datastream/operators/asyncio.md @@ -80,7 +80,7 @@ class AsyncDatabaseRequest extends RichAsyncFunction state; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { state = getRuntimeContext().getState(new ValueStateDescriptor<>("myState", CountWithTimestamp.class)); } diff --git a/docs/content.zh/docs/dev/table/data_stream_api.md b/docs/content.zh/docs/dev/table/data_stream_api.md index ff80243735d36..1b1b958cfa0d3 100644 --- a/docs/content.zh/docs/dev/table/data_stream_api.md +++ b/docs/content.zh/docs/dev/table/data_stream_api.md @@ -1049,7 +1049,7 @@ joinedStream ValueState seen; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { seen = getRuntimeContext().getState( new ValueStateDescriptor<>("seen", String.class)); } diff --git a/docs/content.zh/docs/dev/table/sourcesSinks.md b/docs/content.zh/docs/dev/table/sourcesSinks.md index 7c75eb83ac35c..8d1e437f01743 100644 --- a/docs/content.zh/docs/dev/table/sourcesSinks.md +++ b/docs/content.zh/docs/dev/table/sourcesSinks.md @@ -752,7 +752,7 @@ public class SocketSourceFunction extends RichSourceFunction implements } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { deserializer.open(() -> getRuntimeContext().getMetricGroup()); } diff --git a/docs/content.zh/docs/libs/state_processor_api.md b/docs/content.zh/docs/libs/state_processor_api.md index d36db7c619de6..133c69466ffb1 100644 --- a/docs/content.zh/docs/libs/state_processor_api.md +++ b/docs/content.zh/docs/libs/state_processor_api.md @@ -161,7 +161,7 @@ public class StatefulFunctionWithTime extends KeyedProcessFunction updateTimes; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); @@ -197,7 +197,7 @@ public class ReaderFunction extends KeyedStateReaderFunction updateTimes; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); @@ -430,7 +430,7 @@ public class AccountBootstrapper extends KeyedStateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total",Types.DOUBLE); state = getRuntimeContext().getState(descriptor); } diff --git a/docs/content.zh/docs/try-flink/datastream.md b/docs/content.zh/docs/try-flink/datastream.md index 8d0a0fa36129d..6bc0b289711e9 100644 --- a/docs/content.zh/docs/try-flink/datastream.md +++ b/docs/content.zh/docs/try-flink/datastream.md @@ -377,7 +377,7 @@ public class FraudDetector extends KeyedProcessFunction flagState; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor flagDescriptor = new ValueStateDescriptor<>( "flag", Types.BOOLEAN); @@ -463,7 +463,7 @@ private transient ValueState flagState; private transient ValueState timerState; @Override -public void open(Configuration parameters) { +public void open(OpenContext openContext) { ValueStateDescriptor flagDescriptor = new ValueStateDescriptor<>( "flag", Types.BOOLEAN); @@ -567,7 +567,7 @@ public class FraudDetector extends KeyedProcessFunction timerState; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor flagDescriptor = new ValueStateDescriptor<>( "flag", Types.BOOLEAN); diff --git a/docs/content/docs/dev/datastream/operators/asyncio.md b/docs/content/docs/dev/datastream/operators/asyncio.md index 631c83eaa1b60..6eadf90d3a0c0 100644 --- a/docs/content/docs/dev/datastream/operators/asyncio.md +++ b/docs/content/docs/dev/datastream/operators/asyncio.md @@ -95,7 +95,7 @@ class AsyncDatabaseRequest extends RichAsyncFunction state; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { state = getRuntimeContext().getState(new ValueStateDescriptor<>("myState", CountWithTimestamp.class)); } diff --git a/docs/content/docs/dev/table/data_stream_api.md b/docs/content/docs/dev/table/data_stream_api.md index 8ff7a52ff806b..a5061b5981769 100644 --- a/docs/content/docs/dev/table/data_stream_api.md +++ b/docs/content/docs/dev/table/data_stream_api.md @@ -1047,7 +1047,7 @@ joinedStream ValueState seen; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { seen = getRuntimeContext().getState( new ValueStateDescriptor<>("seen", String.class)); } diff --git a/docs/content/docs/dev/table/sourcesSinks.md b/docs/content/docs/dev/table/sourcesSinks.md index ca0c8f0186b52..2ff0285c2e7fe 100644 --- a/docs/content/docs/dev/table/sourcesSinks.md +++ b/docs/content/docs/dev/table/sourcesSinks.md @@ -835,7 +835,7 @@ public class SocketSourceFunction extends RichSourceFunction implements } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { deserializer.open(() -> getRuntimeContext().getMetricGroup()); } diff --git a/docs/content/docs/libs/state_processor_api.md b/docs/content/docs/libs/state_processor_api.md index 87fc0c62605bf..2d6a45a9cef55 100644 --- a/docs/content/docs/libs/state_processor_api.md +++ b/docs/content/docs/libs/state_processor_api.md @@ -160,7 +160,7 @@ public class StatefulFunctionWithTime extends KeyedProcessFunction updateTimes; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); @@ -196,7 +196,7 @@ public class ReaderFunction extends KeyedStateReaderFunction updateTimes; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor stateDescriptor = new ValueStateDescriptor<>("state", Types.INT); state = getRuntimeContext().getState(stateDescriptor); @@ -429,7 +429,7 @@ public class AccountBootstrapper extends KeyedStateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total",Types.DOUBLE); state = getRuntimeContext().getState(descriptor); } diff --git a/docs/content/docs/try-flink/datastream.md b/docs/content/docs/try-flink/datastream.md index 3f41750011d11..3c7be9905b1ec 100644 --- a/docs/content/docs/try-flink/datastream.md +++ b/docs/content/docs/try-flink/datastream.md @@ -325,7 +325,7 @@ public class FraudDetector extends KeyedProcessFunction flagState; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor flagDescriptor = new ValueStateDescriptor<>( "flag", Types.BOOLEAN); @@ -411,7 +411,7 @@ private transient ValueState flagState; private transient ValueState timerState; @Override -public void open(Configuration parameters) { +public void open(OpenContext openContext) { ValueStateDescriptor flagDescriptor = new ValueStateDescriptor<>( "flag", Types.BOOLEAN); @@ -510,7 +510,7 @@ public class FraudDetector extends KeyedProcessFunction timerState; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor flagDescriptor = new ValueStateDescriptor<>( "flag", Types.BOOLEAN); diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e index 173d6eda87458..5113acebcd81b 100644 --- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e +++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/f7a4e6fa-e7de-48c9-a61e-c13e83f0c72e @@ -797,9 +797,9 @@ Method has parameter of type in (CompactOperator.java:0) Method calls method in (CompactFileUtils.java:117) Method calls method in (PrintTableSinkFactory.java:187) -Method calls method in (PrintTableSinkFactory.java:180) -Method calls method in (PrintTableSinkFactory.java:180) -Method calls method in (PrintTableSinkFactory.java:180) +Method calls method in (PrintTableSinkFactory.java:180) +Method calls method in (PrintTableSinkFactory.java:180) +Method calls method in (PrintTableSinkFactory.java:180) Static Initializer ()> gets field in (CompactCoordinator.java:67) Static Initializer ()> gets field in (CompactorOperator.java:85) Static Initializer ()> calls constructor (int)> in (FileSourceSplitSerializer.java:42) @@ -813,4 +813,4 @@ Static Initializer ()> gets field in (PartitionTimeCommitTrigger.java:51) Static Initializer ()> calls constructor (org.apache.flink.api.common.typeutils.TypeSerializer, org.apache.flink.api.common.typeutils.TypeSerializer)> in (ProcTimeCommitTrigger.java:46) Static Initializer ()> gets field in (ProcTimeCommitTrigger.java:46) -Static Initializer ()> gets field in (ProcTimeCommitTrigger.java:46) \ No newline at end of file +Static Initializer ()> gets field in (ProcTimeCommitTrigger.java:46) diff --git a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java index 438ff218e6315..db25079395d9e 100644 --- a/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java +++ b/flink-connectors/flink-connector-base/src/test/java/org/apache/flink/connector/base/source/reader/CoordinatedSourceITCase.java @@ -20,13 +20,13 @@ import org.apache.flink.api.common.accumulators.ListAccumulator; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; import org.apache.flink.api.connector.source.mocks.MockSourceSplit; -import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.base.source.reader.mocks.MockBaseSource; import org.apache.flink.connector.base.source.reader.mocks.MockSplitEnumerator; import org.apache.flink.streaming.api.datastream.DataStream; @@ -104,7 +104,7 @@ private void executeAndVerify( stream.addSink( new RichSinkFunction() { @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { getRuntimeContext() .addAccumulator("result", new ListAccumulator()); } diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSink.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSink.java index ea1c6cb13bb9f..e3fc5b2a16f63 100644 --- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSink.java +++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSink.java @@ -18,7 +18,7 @@ package org.apache.flink.connector.file.table.batch.compact; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.connector.file.table.FileSystemCommitter; import org.apache.flink.connector.file.table.FileSystemFactory; import org.apache.flink.connector.file.table.PartitionCommitPolicy; @@ -81,7 +81,7 @@ public BatchPartitionCommitterSink( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { partitionsFiles = new HashMap<>(); } diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSinkTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSinkTest.java index db7e96b7d6c4c..688a7ec80a72a 100644 --- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSinkTest.java +++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchPartitionCommitterSinkTest.java @@ -18,8 +18,8 @@ package org.apache.flink.connector.file.table.batch.compact; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.configuration.Configuration; import org.apache.flink.connector.file.table.FileSystemCommitterTest; import org.apache.flink.connector.file.table.FileSystemFactory; import org.apache.flink.connector.file.table.PartitionCommitPolicyFactory; @@ -77,7 +77,7 @@ public void testPartitionCommit() throws Exception { new LinkedHashMap<>(), identifier, new PartitionCommitPolicyFactory(null, null, null, null)); - committerSink.open(new Configuration()); + committerSink.open(DefaultOpenContext.INSTANCE); List pathList1 = createFiles(path, "task-1/p1=0/p2=0/", "f1", "f2"); List pathList2 = createFiles(path, "task-2/p1=0/p2=0/", "f3"); diff --git a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/TestLockTableSinkFactory.java b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/TestLockTableSinkFactory.java index 142108dd65724..d27721aa2591a 100644 --- a/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/TestLockTableSinkFactory.java +++ b/flink-connectors/flink-connector-hive/src/test/java/org/apache/flink/table/catalog/hive/factories/TestLockTableSinkFactory.java @@ -18,8 +18,8 @@ package org.apache.flink.table.catalog.hive.factories; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkFunction; import org.apache.flink.table.catalog.CatalogLock; @@ -127,7 +127,7 @@ private TestLockSink(ObjectIdentifier tableIdentifier, CatalogLock.Factory lockF } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.lock = lockFactory.create(); } diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java index 7b3ad761cf504..766156ce81609 100644 --- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java +++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopMapFunction.java @@ -19,6 +19,8 @@ package org.apache.flink.hadoopcompatibility.mapred; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter; @@ -26,7 +28,6 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector; import org.apache.flink.util.Collector; import org.apache.flink.util.InstantiationUtil; @@ -83,9 +84,10 @@ public HadoopMapFunction(Mapper hadoopMapper, this.jobConf = conf; } + @PublicEvolving @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.mapper.configure(jobConf); this.reporter = new HadoopDummyReporter(); diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java index 525085dac1f04..4db71d76d7b8f 100644 --- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java +++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java @@ -19,7 +19,9 @@ package org.apache.flink.hadoopcompatibility.mapred; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -28,7 +30,6 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector; import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator; import org.apache.flink.util.Collector; @@ -105,9 +106,10 @@ public HadoopReduceCombineFunction( } @SuppressWarnings("unchecked") + @PublicEvolving @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.reducer.configure(jobConf); this.combiner.configure(jobConf); diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java index 73ac575a2de36..441be6d344653 100644 --- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java +++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java @@ -19,6 +19,8 @@ package org.apache.flink.hadoopcompatibility.mapred; import org.apache.flink.annotation.Public; +import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -27,7 +29,6 @@ import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.TupleTypeInfo; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector; import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator; import org.apache.flink.util.Collector; @@ -89,9 +90,10 @@ public HadoopReduceFunction( } @SuppressWarnings("unchecked") + @PublicEvolving @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.reducer.configure(jobConf); this.reporter = new HadoopDummyReporter(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java b/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java index aa5d1c1776343..cd03785457b8f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/aggregators/Aggregator.java @@ -46,7 +46,7 @@ * public class MyFilter extends FilterFunction<Double> { * private LongSumAggregator agg; * - * public void open(Configuration parameters) { + * public void open(OpenContext openContext) { * agg = getIterationRuntimeContext().getIterationAggregator("numFiltered"); * } * diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java index 32ac59205429d..7a25c4abc7206 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/AbstractRichFunction.java @@ -25,8 +25,8 @@ /** * An abstract stub implementation for rich user-defined functions. Rich functions have additional - * methods for initialization ({@link #open(Configuration)}) and teardown ({@link #close()}), as - * well as access to their runtime execution context via {@link #getRuntimeContext()}. + * methods for initialization ({@link #open(OpenContext)}) and teardown ({@link #close()}), as well + * as access to their runtime execution context via {@link #getRuntimeContext()}. */ @Public public abstract class AbstractRichFunction implements RichFunction, Serializable { diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichAggregateFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichAggregateFunction.java index 1cc8a243e7213..f0f30dc185592 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichAggregateFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichAggregateFunction.java @@ -23,8 +23,7 @@ /** * Rich variant of the {@link AggregateFunction}. As a {@link RichFunction}, it gives access to the * {@link RuntimeContext} and provides setup and teardown methods: {@link - * RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @see AggregateFunction * @param The type of the values that are aggregated (input values) diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java index e2bb879180edf..1e5443ee2eca4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCoGroupFunction.java @@ -24,8 +24,7 @@ /** * Rich variant of the {@link CoGroupFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The type of the elements in the first input. * @param The type of the elements in the second input. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java index b95ab8328b9f1..bdbebfd2c3642 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichCrossFunction.java @@ -23,8 +23,7 @@ /** * Rich variant of the {@link CrossFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The type of the elements in the first input. * @param The type of the elements in the second input. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java index 1a26a2a9e3c58..614f30b18b42e 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFilterFunction.java @@ -23,8 +23,7 @@ /** * Rich variant of the {@link FilterFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The type of the filtered elements. */ diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java index e8d0a9d849f8d..de12313a55310 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatJoinFunction.java @@ -24,8 +24,7 @@ /** * Rich variant of the {@link FlatJoinFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The type of the elements in the first input. * @param The type of the elements in the second input. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java index dab566ddbae54..e8b3ec0f2e6a5 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichFlatMapFunction.java @@ -24,8 +24,7 @@ /** * Rich variant of the {@link FlatMapFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param Type of the input elements. * @param Type of the returned elements. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupCombineFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupCombineFunction.java index 32769f3344a92..e025ad9a3fb03 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupCombineFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupCombineFunction.java @@ -24,8 +24,7 @@ /** * Rich variant of the {@link GroupCombineFunction}. As a {@link RichFunction}, it gives access to * the {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The data type of the elements to be combined. * @param The resulting data type of the elements to be combined. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java index df18a99114459..f9f1dd8981af4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichGroupReduceFunction.java @@ -24,8 +24,7 @@ /** * Rich variant of the {@link GroupReduceFunction}. As a {@link RichFunction}, it gives access to * the {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * *

Partial computation can significantly improve the performance of a {@link * RichGroupReduceFunction}. This technique is also known as applying a Combiner. Implement the diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java index 53103104c1667..f488747c00392 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichJoinFunction.java @@ -23,8 +23,7 @@ /** * Rich variant of the {@link JoinFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The type of the elements in the first input. * @param The type of the elements in the second input. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java index 33880164d64a7..098e2c8aa5e58 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapFunction.java @@ -23,8 +23,7 @@ /** * Rich variant of the {@link MapFunction}. As a {@link RichFunction}, it gives access to the {@link * org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown methods: - * {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param Type of the input elements. * @param Type of the returned elements. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java index 24469c0f82f8e..bcfbe39aecabf 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichMapPartitionFunction.java @@ -24,8 +24,7 @@ /** * Rich variant of the {@link MapPartitionFunction}. As a {@link RichFunction}, it gives access to * the {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param Type of the input elements. * @param Type of the returned elements. diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java index 18a8c9085e011..4e71b841b1c91 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/RichReduceFunction.java @@ -23,8 +23,7 @@ /** * Rich variant of the {@link ReduceFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param Type of the elements that this function processes. */ diff --git a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java index 046fddf1c3a72..e71479fe3eb82 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/functions/util/FunctionUtils.java @@ -20,18 +20,18 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.configuration.Configuration; /** Utility class that contains helper methods to work with Flink {@link Function} class. */ @Internal public final class FunctionUtils { - public static void openFunction(Function function, Configuration parameters) throws Exception { + public static void openFunction(Function function, OpenContext openContext) throws Exception { if (function instanceof RichFunction) { RichFunction richFunction = (RichFunction) function; - richFunction.open(parameters); + richFunction.open(openContext); } } diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java index f8a81e79fc104..93fb78b4ec2ff 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/BulkIterationBase.java @@ -26,6 +26,7 @@ import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.operators.IterationOperator; import org.apache.flink.api.common.operators.Operator; @@ -35,7 +36,6 @@ import org.apache.flink.api.common.operators.util.UserCodeClassWrapper; import org.apache.flink.api.common.operators.util.UserCodeWrapper; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; import org.apache.flink.types.LongValue; import org.apache.flink.util.Collector; import org.apache.flink.util.Visitor; @@ -238,7 +238,7 @@ public static class TerminationCriterionMapper extends AbstractRichFunction private TerminationCriterionAggregator aggregator; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { aggregator = getIterationRuntimeContext() .getIterationAggregator(TERMINATION_CRITERION_AGGREGATOR_NAME); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java index 0364e998acdd5..cdd85dcc959da 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupOperatorBase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; @@ -295,7 +296,7 @@ protected List executeOnCollections( CoGroupFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); List result = new ArrayList(); Collector resultCollector = diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupRawOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupRawOperatorBase.java index c98dc1bd9c3c2..89d982654b04f 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupRawOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CoGroupRawOperatorBase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -217,7 +218,7 @@ protected List executeOnCollections( CoGroupFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); List result = new ArrayList(); Collector resultCollector = diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java index 9b75541b25bee..bf09a281936a9 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/CrossOperatorBase.java @@ -22,6 +22,7 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.CrossFunction; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.BinaryOperatorInformation; @@ -98,7 +99,7 @@ protected List executeOnCollections( CrossFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(inputData1.size() * inputData2.size()); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java index e1430083ac5f5..d56551dae33ac 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FilterOperatorBase.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -59,7 +60,7 @@ protected List executeOnCollections( FlatMapFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(inputData.size()); ListCollector collector = new ListCollector(result); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java index be309d9756447..974e016428f39 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/FlatMapOperatorBase.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; @@ -62,7 +63,7 @@ protected List executeOnCollections( FlatMapFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(input.size()); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupCombineOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupCombineOperatorBase.java index 97c0f34cc8592..7cb4e91e0dff4 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupCombineOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupCombineOperatorBase.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; @@ -140,7 +141,7 @@ public int compare(IN o1, IN o2) { } FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java index cd6449c4ef495..6d75e0f33c9c7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorBase.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.Partitioner; @@ -222,7 +223,7 @@ public int compare(IN o1, IN o2) { } FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java index 1bd8bd7e989e9..5a70fe4b92a45 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBase.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; @@ -88,7 +89,7 @@ protected List executeOnCollections( FlatJoinFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, runtimeContext); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); TypeInformation leftInformation = getOperatorInfo().getFirstInputType(); TypeInformation rightInformation = getOperatorInfo().getSecondInputType(); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java index e0a6fc1525d58..47545fa9e48e7 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapOperatorBase.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; @@ -65,7 +66,7 @@ protected List executeOnCollections( MapFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(inputData.size()); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java index 64a681311a865..e03ff7fec5b12 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/MapPartitionOperatorBase.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.MapPartitionFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingIterator; @@ -68,7 +69,7 @@ protected List executeOnCollections( MapPartitionFunction function = this.userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); ArrayList result = new ArrayList(inputData.size() / 4); diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java index 7c22e1c91e64d..48da58e265192 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBase.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.CopyingListCollector; @@ -135,7 +136,7 @@ protected List executeOnCollections( FlatJoinFunction function = userFunction.getUserCodeObject(); FunctionUtils.setFunctionRuntimeContext(function, runtimeContext); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); List result = new ArrayList<>(); Collector collector = diff --git a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java index e00c9f1612053..9edbb96027268 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/base/ReduceOperatorBase.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.InvalidProgramException; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -219,7 +220,7 @@ protected List executeOnCollections( } FunctionUtils.setFunctionRuntimeContext(function, ctx); - FunctionUtils.openFunction(function, this.parameters); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); TypeSerializer serializer = getOperatorInfo().getInputType().createSerializer(executionConfig); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java index bb02c3dbf9533..d2cb3d287ecf6 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/FlatMapOperatorCollectionTest.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.util.Collector; @@ -103,7 +103,7 @@ public class IdRichFlatMap extends RichFlatMapFunction { private boolean isClosed = false; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { isOpened = true; RuntimeContext ctx = getRuntimeContext(); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java index 908963c16e82d..32aa85116b1b7 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/InnerJoinOperatorBaseTest.java @@ -22,11 +22,11 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.util.Collector; @@ -104,7 +104,7 @@ public void testJoinRich() { final RichFlatJoinFunction joiner = new RichFlatJoinFunction() { @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened.compareAndSet(false, true); assertEquals(0, getRuntimeContext().getIndexOfThisSubtask()); assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java index ccfb2527ca87f..40adaec04b292 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/MapOperatorTest.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; @@ -92,7 +92,7 @@ public void testMapWithRuntimeContext() { new RichMapFunction() { @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened.set(true); RuntimeContext ctx = getRuntimeContext(); assertEquals(0, ctx.getIndexOfThisSubtask()); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java index c0c655868f44f..4f35dab35f332 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/OuterJoinOperatorBaseTest.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.FlatJoinFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.operators.BinaryOperatorInformation; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.util.Collector; @@ -260,7 +260,7 @@ private static class MockRichFlatJoinFunction final AtomicBoolean closed = new AtomicBoolean(false); @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened.compareAndSet(false, true); assertEquals(0, getRuntimeContext().getIndexOfThisSubtask()); assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); diff --git a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java index 34d6406e7686f..70b5d6a8f1084 100644 --- a/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/common/operators/base/PartitionMapOperatorTest.java @@ -22,12 +22,12 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.MapPartitionFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; import org.apache.flink.api.common.operators.UnaryOperatorInformation; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.util.Collector; @@ -59,7 +59,7 @@ public void testMapPartitionWithRuntimeContext() { new RichMapPartitionFunction() { @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened.set(true); RuntimeContext ctx = getRuntimeContext(); assertEquals(0, ctx.getIndexOfThisSubtask()); diff --git a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java index 3faa149ad12fe..f5a9ca3a890e1 100644 --- a/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java +++ b/flink-core/src/test/java/org/apache/flink/api/java/typeutils/TypeExtractorTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.functions.InvalidTypesException; import org.apache.flink.api.common.functions.JoinFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichCoGroupFunction; import org.apache.flink.api.common.functions.RichCrossFunction; import org.apache.flink.api.common.functions.RichFlatJoinFunction; @@ -46,7 +47,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple9; -import org.apache.flink.configuration.Configuration; import org.apache.flink.types.DoubleValue; import org.apache.flink.types.Either; import org.apache.flink.types.IntValue; @@ -1821,7 +1821,7 @@ public void testFunction() { public void setRuntimeContext(RuntimeContext t) {} @Override - public void open(Configuration parameters) throws Exception {} + public void open(OpenContext openContext) throws Exception {} @Override public RuntimeContext getRuntimeContext() { diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/BlockingIncrementingMapFunction.java b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/BlockingIncrementingMapFunction.java index 0214b5f4abc25..b14e1467e8afb 100644 --- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/BlockingIncrementingMapFunction.java +++ b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/BlockingIncrementingMapFunction.java @@ -19,9 +19,9 @@ package org.apache.flink.batch.tests; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.batch.tests.util.FileBasedOneShotLatch; -import org.apache.flink.configuration.Configuration; import java.nio.file.Paths; @@ -43,7 +43,7 @@ public BlockingIncrementingMapFunction(final String latchFilePath) { } @Override - public void open(final Configuration parameters) { + public void open(final OpenContext openContext) { latch = new FileBasedOneShotLatch(Paths.get(latchFilePath)); } diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java index 7c2dc99e8eb6e..943863d05cfae 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SemanticsCheckMapper.java @@ -18,10 +18,10 @@ package org.apache.flink.streaming.tests; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.io.Serializable; @@ -62,7 +62,7 @@ public void flatMap(Event event, Collector out) throws Exception { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor sequenceStateDescriptor = new ValueStateDescriptor<>("sequenceState", Long.class); diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java index efd339fee9e84..5235ca4dea211 100644 --- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java +++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/SlidingWindowCheckMapper.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.tests; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; @@ -26,7 +27,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.io.IOException; @@ -59,7 +59,7 @@ public class SlidingWindowCheckMapper } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor>> previousWindowDescriptor = new ValueStateDescriptor<>( "eventsSeenSoFar", diff --git a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java index b157de01694f1..d9360bc361174 100644 --- a/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java +++ b/flink-end-to-end-tests/flink-netty-shuffle-memory-control-test/src/main/java/org/apache/flink/streaming/tests/NettyShuffleMemoryControlTestProgram.java @@ -18,10 +18,10 @@ package org.apache.flink.streaming.tests; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; @@ -121,7 +121,7 @@ public StringSourceFunction(long runningTimeInSeconds) { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { isRunning = true; stopTime = System.nanoTime() + runningTimeInSeconds * 1_000_000_000L; } diff --git a/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateProducer.java b/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateProducer.java index 149386669b688..252dcb8803104 100644 --- a/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateProducer.java +++ b/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateProducer.java @@ -17,6 +17,7 @@ package org.apache.flink.streaming.tests.queryablestate; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.MapState; @@ -25,7 +26,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; @@ -110,8 +110,8 @@ private static class EmailSource extends RichSourceFunction { private transient Random random; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.random = new Random(); this.isRunning = true; } @@ -158,7 +158,7 @@ private static class TestFlatMap extends RichFlatMapFunction private transient long lastCompletedCheckpoint; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { MapStateDescriptor stateDescriptor = new MapStateDescriptor<>( QsConstants.STATE_NAME, diff --git a/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/src/main/java/org/apache/flink/streaming/tests/RocksDBStateMemoryControlTestProgram.java b/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/src/main/java/org/apache/flink/streaming/tests/RocksDBStateMemoryControlTestProgram.java index b45d2bc6a5945..62d114ded7e91 100644 --- a/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/src/main/java/org/apache/flink/streaming/tests/RocksDBStateMemoryControlTestProgram.java +++ b/flink-end-to-end-tests/flink-rocksdb-state-memory-control-test/src/main/java/org/apache/flink/streaming/tests/RocksDBStateMemoryControlTestProgram.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.tests; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -28,7 +29,6 @@ import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.KeyedStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.windowing.WindowFunction; @@ -113,7 +113,7 @@ private static class ValueStateMapper extends RichMapFunction { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { int index = getRuntimeContext().getIndexOfThisSubtask(); if (useValueState) { valueState = @@ -151,7 +151,7 @@ private static class ListStateMapper extends RichMapFunction { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { int index = getRuntimeContext().getIndexOfThisSubtask(); if (useListState) { listState = @@ -184,7 +184,7 @@ private static class MapStateMapper extends RichMapFunction { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { int index = getRuntimeContext().getIndexOfThisSubtask(); if (useMapState) { mapState = diff --git a/flink-end-to-end-tests/flink-state-evolution-test/src/main/java/org/apache/flink/test/StatefulStreamingJob.java b/flink-end-to-end-tests/flink-state-evolution-test/src/main/java/org/apache/flink/test/StatefulStreamingJob.java index 55cd5ffce692e..94a7778a1bdf1 100644 --- a/flink-end-to-end-tests/flink-state-evolution-test/src/main/java/org/apache/flink/test/StatefulStreamingJob.java +++ b/flink-end-to-end-tests/flink-state-evolution-test/src/main/java/org/apache/flink/test/StatefulStreamingJob.java @@ -18,6 +18,7 @@ package org.apache.flink.test; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ValueState; @@ -27,7 +28,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.utils.ParameterTool; import org.apache.flink.avro.generated.Address; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; @@ -106,8 +106,8 @@ public static class MyStatefulFunction extends RichMapFunction private transient ValueState> eitherState; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.avroState = getRuntimeContext().getState(AVRO_DESCRIPTOR); this.tupleState = getRuntimeContext().getState(TUPLE_DESCRIPTOR); this.eitherState = getRuntimeContext().getState(EITHER_DESCRIPTOR); diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java index 0c6525fa0d2e1..a5aae3d90772a 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java @@ -19,6 +19,7 @@ package org.apache.flink.examples.java.clustering; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.DataSet; @@ -28,7 +29,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; import org.apache.flink.examples.java.clustering.util.KMeansData; import org.slf4j.Logger; @@ -258,7 +258,7 @@ public static final class SelectNearestCenter /** Reads the centroid values from a broadcast variable into a collection. */ @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); } diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java index 4329c43080c72..d370148c27149 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java @@ -19,6 +19,7 @@ package org.apache.flink.examples.java.distcp; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -27,7 +28,6 @@ import org.apache.flink.api.java.operators.FlatMapOperator; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FSDataInputStream; import org.apache.flink.core.fs.FSDataOutputStream; import org.apache.flink.core.fs.FileStatus; @@ -119,7 +119,7 @@ public static void main(String[] args) throws Exception { private LongCounter bytesCounter; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { bytesCounter = getRuntimeContext().getLongCounter(BYTES_COPIED_CNT_NAME); fileCounter = diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java index 8eaf3c52389bd..e493a65285c95 100644 --- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java +++ b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/relational/EmptyFieldsCountAccumulator.java @@ -20,12 +20,12 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -149,8 +149,8 @@ public static final class EmptyFieldFilter extends RichFilterFunction, List< } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { // When multiple instances of this class and JCuda exist in different class loaders, // then we will get UnsatisfiedLinkError. // To avoid that, we need to temporarily override the java.io.tmpdir, where the JCuda diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java index 1f43021a6b747..82ec367417814 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/statemachine/StateMachineExample.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.examples.statemachine; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.serialization.SimpleStringEncoder; import org.apache.flink.api.common.state.ValueState; @@ -26,7 +27,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; import org.apache.flink.api.java.utils.ParameterTool; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.MemorySize; import org.apache.flink.connector.datagen.source.DataGeneratorSource; import org.apache.flink.connector.datagen.source.GeneratorFunction; @@ -195,7 +195,7 @@ static class StateMachineMapper extends RichFlatMapFunction { private ValueState currentState; @Override - public void open(Configuration conf) { + public void open(OpenContext openContext) { // get access to the state object currentState = getRuntimeContext().getState(new ValueStateDescriptor<>("state", State.class)); diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java index 728a4848e4908..4d5e9a0ecfefc 100644 --- a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java +++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSinkFunction.java @@ -18,8 +18,8 @@ package org.apache.flink.formats.protobuf.table; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.table.data.RowData; @@ -32,8 +32,8 @@ public TestProtobufSinkFunction(SerializationSchema serializer) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.serializer.open(null); } diff --git a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java index e084a5f902ab5..419f298c0a256 100644 --- a/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java +++ b/flink-formats/flink-protobuf/src/test/java/org/apache/flink/formats/protobuf/table/TestProtobufSourceFunction.java @@ -18,8 +18,8 @@ package org.apache.flink.formats.protobuf.table; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.serialization.DeserializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.table.data.RowData; @@ -32,8 +32,8 @@ public TestProtobufSourceFunction(DeserializationSchema deserializer) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.deserializer.open(null); } diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java index 8bed2a654023f..f02d0f792df32 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingITCase.java @@ -18,13 +18,13 @@ package org.apache.flink.hdfstests; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.io.FilePathFilter; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -214,7 +214,7 @@ private static class TestingSinkFunction extends RichSinkFunction { private transient Comparator comparator; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { // this sink can only work with DOP 1 assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); diff --git a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java index 480df52813baf..c84cbaa82b6d9 100644 --- a/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java +++ b/flink-fs-tests/src/test/java/org/apache/flink/hdfstests/ContinuousFileProcessingTest.java @@ -19,6 +19,7 @@ package org.apache.flink.hdfstests; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FilePathFilter; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -26,7 +27,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileInputSplit; import org.apache.flink.core.fs.Path; import org.apache.flink.core.testutils.OneShotLatch; @@ -636,7 +636,7 @@ public boolean filterPath(Path filePath) { final FileVerifyingSourceContext context = new FileVerifyingSourceContext(new OneShotLatch(), monitoringFunction); - monitoringFunction.open(new Configuration()); + monitoringFunction.open(DefaultOpenContext.INSTANCE); monitoringFunction.run(context); Assert.assertArrayEquals(filesKept.toArray(), context.getSeenFiles().toArray()); @@ -697,7 +697,7 @@ public void testNestedFilesProcessing() throws Exception { final FileVerifyingSourceContext context = new FileVerifyingSourceContext(new OneShotLatch(), monitoringFunction); - monitoringFunction.open(new Configuration()); + monitoringFunction.open(DefaultOpenContext.INSTANCE); monitoringFunction.run(context); Assert.assertArrayEquals(filesToBeRead.toArray(), context.getSeenFiles().toArray()); @@ -737,7 +737,7 @@ public void testSortingOnModTime() throws Exception { ModTimeVerifyingSourceContext context = new ModTimeVerifyingSourceContext(modTimes); - monitoringFunction.open(new Configuration()); + monitoringFunction.open(DefaultOpenContext.INSTANCE); monitoringFunction.run(context); Assert.assertEquals(splits.length, context.getCounter()); @@ -777,7 +777,7 @@ public void testProcessOnce() throws Exception { @Override public void run() { try { - monitoringFunction.open(new Configuration()); + monitoringFunction.open(DefaultOpenContext.INSTANCE); monitoringFunction.run(context); // we would never arrive here if we were in @@ -937,7 +937,7 @@ public void testProcessContinuously() throws Exception { @Override public void run() { try { - monitoringFunction.open(new Configuration()); + monitoringFunction.open(DefaultOpenContext.INSTANCE); monitoringFunction.run(context); } catch (Exception e) { Assert.fail(e.getMessage()); diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java index dbccd080e1ab5..59d09ef927122 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/AggregateOperator.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; @@ -35,7 +36,6 @@ import org.apache.flink.api.java.aggregation.Aggregations; import org.apache.flink.api.java.tuple.Tuple; import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.apache.flink.util.Preconditions; @@ -283,7 +283,7 @@ public AggregatingUdf(AggregationFunction[] aggFunctions, int[] fieldPos } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { for (AggregationFunction aggFunction : aggFunctions) { aggFunction.initializeAggregate(); } diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java index 00e9d3befb88f..958b8c85fba2a 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/UdfOperator.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.Public; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.operators.SemanticProperties; import org.apache.flink.api.java.DataSet; import org.apache.flink.configuration.Configuration; @@ -49,7 +50,7 @@ public interface UdfOperator> { /** * Gets the configuration parameters that will be passed to the UDF's open method {@link - * org.apache.flink.api.common.functions.AbstractRichFunction#open(Configuration)}. The + * org.apache.flink.api.common.functions.AbstractRichFunction#open(OpenContext)}. The * configuration is set via the {@link #withParameters(Configuration)} method. * * @return The configuration parameters for the UDF. @@ -80,7 +81,7 @@ public interface UdfOperator> { /** * Sets the configuration parameters for the UDF. These are optional parameters that are passed * to the UDF in the {@link - * org.apache.flink.api.common.functions.AbstractRichFunction#open(Configuration)} method. + * org.apache.flink.api.common.functions.AbstractRichFunction#open(OpenContext)} method. * * @param parameters The configuration parameters for the UDF. * @return The operator itself, to allow chaining function calls. diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java index f49daa002ce13..ab45f6fcccfe6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/RichCombineToGroupCombineWrapper.java @@ -21,9 +21,9 @@ import org.apache.flink.api.common.functions.CombineFunction; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.apache.flink.util.Preconditions; @@ -43,9 +43,9 @@ public RichCombineToGroupCombineWrapper(F wrappedFunction) { } @Override - public void open(Configuration config) throws Exception { + public void open(OpenContext openContext) throws Exception { wrappedFunction.setRuntimeContext(getRuntimeContext()); - wrappedFunction.open(config); + wrappedFunction.open(openContext); } @Override diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java index b7b7394ff3aa7..8ff6fa92725c9 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/WrappingFunction.java @@ -21,9 +21,9 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; /** * Wrapper around {@link Function}. @@ -42,8 +42,8 @@ protected WrappingFunction(T wrappedFunction) { } @Override - public void open(Configuration parameters) throws Exception { - FunctionUtils.openFunction(this.wrappedFunction, parameters); + public void open(OpenContext openContext) throws Exception { + FunctionUtils.openFunction(this.wrappedFunction, openContext); } @Override diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java index 0198bb1d5f51e..1782f7c515bd6 100644 --- a/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java +++ b/flink-java/src/main/java/org/apache/flink/api/java/utils/DataSetUtils.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.distributions.DataDistribution; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.MapPartitionFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.api.common.operators.Keys; @@ -43,7 +44,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; import org.apache.flink.api.java.typeutils.TypeExtractor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.AbstractID; import org.apache.flink.util.Collector; @@ -107,8 +107,8 @@ public static DataSet> zipWithIndex(DataSet input) { long start = 0; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); List> offsets = getRuntimeContext() @@ -206,8 +206,8 @@ public static DataSet> zipWithUniqueId(DataSet input) { long label = 0; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); shifter = getBitSize(getRuntimeContext().getNumberOfParallelSubtasks() - 1); taskId = getRuntimeContext().getIndexOfThisSubtask(); } diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java index 48fd251bad63d..7e8f3f0a87b14 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionAccumulatorsTest.java @@ -20,10 +20,10 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.configuration.Configuration; import org.junit.jupiter.api.Test; @@ -63,7 +63,7 @@ private static class CountingMapper extends RichMapFunction { private IntCounter accumulator; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { accumulator = getRuntimeContext().getIntCounter(ACCUMULATOR_NAME); } diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java index b7baedba03a33..354f78bb9fa2f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/CollectionExecutionWithBroadcastVariableTest.java @@ -18,12 +18,12 @@ package org.apache.flink.api.common.operators; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichCrossFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; -import org.apache.flink.configuration.Configuration; import org.junit.jupiter.api.Test; @@ -100,7 +100,7 @@ private static final class SuffixAppender extends RichMapFunctiongetBroadcastVariable(BC_VAR_NAME).get(0); } @@ -115,7 +115,7 @@ private static final class SuffixCross extends RichCrossFunctiongetBroadcastVariable(BC_VAR_NAME).get(0); } diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java index 2f0a18bb329b5..59e87bf2d672f 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/CoGroupOperatorCollectionTest.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.functions.CoGroupFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichCoGroupFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; @@ -30,7 +31,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.builder.Tuple2Builder; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.util.Collector; @@ -158,7 +158,7 @@ private static class SumCoGroup private boolean isClosed = false; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { isOpened = true; RuntimeContext ctx = getRuntimeContext(); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java index ada8c5d6e82cc..441d76bfa8ef1 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/GroupReduceOperatorTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.RuntimeUDFContext; @@ -28,7 +29,6 @@ import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.apache.flink.util.Collector; @@ -143,7 +143,7 @@ public void reduce( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened.set(true); RuntimeContext ctx = getRuntimeContext(); assertThat(ctx.getIndexOfThisSubtask()).isZero(); diff --git a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java index 69dfdf08a71b5..eb7fa4fe94869 100644 --- a/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java +++ b/flink-java/src/test/java/org/apache/flink/api/common/operators/base/ReduceOperatorTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; @@ -29,7 +30,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.operators.ReduceOperator; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; import org.junit.jupiter.api.Test; @@ -117,7 +117,7 @@ public Tuple2 reduce( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened.set(true); RuntimeContext ctx = getRuntimeContext(); assertThat(ctx.getIndexOfThisSubtask()).isZero(); diff --git a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java index b85827857b68c..63322a1b0b396 100644 --- a/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java +++ b/flink-kubernetes/src/test/java/org/apache/flink/kubernetes/highavailability/KubernetesHighAvailabilityRecoverFromSavepointITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -179,8 +180,8 @@ private JobGraph createJobGraph(File stateBackendFolder) throws Exception { ValueState state; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total", Types.INT); diff --git a/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/PatternStreamScalaJavaAPIInteroperabilityTest.scala b/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/PatternStreamScalaJavaAPIInteroperabilityTest.scala index a8f1921bce788..c4945a04fd024 100644 --- a/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/PatternStreamScalaJavaAPIInteroperabilityTest.scala +++ b/flink-libraries/flink-cep-scala/src/test/scala/org/apache/flink/cep/scala/PatternStreamScalaJavaAPIInteroperabilityTest.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.cep.scala -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{DefaultOpenContext, OpenContext, RuntimeContext} import org.apache.flink.api.common.functions.util.{FunctionUtils, ListCollector} import org.apache.flink.cep.functions.{PatternProcessFunction, TimedOutPartialMatchHandler} import org.apache.flink.cep.operator.CepOperator @@ -138,7 +138,7 @@ class PatternStreamScalaJavaAPIInteroperabilityTest extends TestLogger { val fun = oper.getUserFunction FunctionUtils.setFunctionRuntimeContext(fun, Mockito.mock(classOf[RuntimeContext])) - FunctionUtils.openFunction(fun, new Configuration()) + FunctionUtils.openFunction(fun, DefaultOpenContext.INSTANCE) fun } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternFlatSelectFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternFlatSelectFunction.java index 3a1fe3417645d..bf5626067c79e 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternFlatSelectFunction.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternFlatSelectFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.cep; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.util.Collector; @@ -28,8 +29,7 @@ /** * Rich variant of the {@link PatternFlatSelectFunction}. As a {@link RichFunction}, it gives access * to the {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and - * teardown methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and - * {@link RichFunction#close()}. + * teardown methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param Type of the input elements * @param Type of the output element diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternSelectFunction.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternSelectFunction.java index fb279ecc3d621..98fb9b2d71b23 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternSelectFunction.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/RichPatternSelectFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.cep; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import java.util.List; @@ -27,8 +28,7 @@ /** * Rich variant of the {@link PatternSelectFunction}. As a {@link RichFunction}, it gives access to * the {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param Type of the input elements * @param Type of the output element diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternFlatSelectAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternFlatSelectAdapter.java index 6b8327688031c..cafe122dba684 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternFlatSelectAdapter.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternFlatSelectAdapter.java @@ -19,10 +19,10 @@ package org.apache.flink.cep.functions.adaptors; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.cep.PatternFlatSelectFunction; import org.apache.flink.cep.functions.PatternProcessFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.util.List; @@ -41,9 +41,9 @@ public PatternFlatSelectAdapter(final PatternFlatSelectFunction flatSel } @Override - public void open(final Configuration parameters) throws Exception { + public void open(final OpenContext openContext) throws Exception { FunctionUtils.setFunctionRuntimeContext(flatSelectFunction, getRuntimeContext()); - FunctionUtils.openFunction(flatSelectFunction, parameters); + FunctionUtils.openFunction(flatSelectFunction, openContext); } @Override diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternSelectAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternSelectAdapter.java index ff62b32d064c4..007c41bea6776 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternSelectAdapter.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternSelectAdapter.java @@ -19,10 +19,10 @@ package org.apache.flink.cep.functions.adaptors; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.cep.PatternSelectFunction; import org.apache.flink.cep.functions.PatternProcessFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.util.List; @@ -41,9 +41,9 @@ public PatternSelectAdapter(final PatternSelectFunction selectFunction) } @Override - public void open(final Configuration parameters) throws Exception { + public void open(final OpenContext openContext) throws Exception { FunctionUtils.setFunctionRuntimeContext(selectFunction, getRuntimeContext()); - FunctionUtils.openFunction(selectFunction, parameters); + FunctionUtils.openFunction(selectFunction, openContext); } @Override diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutFlatSelectAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutFlatSelectAdapter.java index 482a2dce8fb66..3277d08fb7d98 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutFlatSelectAdapter.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutFlatSelectAdapter.java @@ -19,12 +19,12 @@ package org.apache.flink.cep.functions.adaptors; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.cep.PatternFlatSelectFunction; import org.apache.flink.cep.PatternFlatTimeoutFunction; import org.apache.flink.cep.functions.PatternProcessFunction; import org.apache.flink.cep.functions.TimedOutPartialMatchHandler; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.apache.flink.util.OutputTag; @@ -56,10 +56,10 @@ public PatternTimeoutFlatSelectAdapter( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); FunctionUtils.setFunctionRuntimeContext(flatTimeoutFunction, getRuntimeContext()); - FunctionUtils.openFunction(flatTimeoutFunction, parameters); + FunctionUtils.openFunction(flatTimeoutFunction, openContext); if (sideCollector == null) { sideCollector = new SideCollector<>(checkNotNull(timedOutPartialMatchesTag)); diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutSelectAdapter.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutSelectAdapter.java index 9362c6e75b8a6..90aafef7621d5 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutSelectAdapter.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/functions/adaptors/PatternTimeoutSelectAdapter.java @@ -19,12 +19,12 @@ package org.apache.flink.cep.functions.adaptors; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.cep.PatternSelectFunction; import org.apache.flink.cep.PatternTimeoutFunction; import org.apache.flink.cep.functions.PatternProcessFunction; import org.apache.flink.cep.functions.TimedOutPartialMatchHandler; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.OutputTag; import java.util.List; @@ -53,10 +53,10 @@ public PatternTimeoutSelectAdapter( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); FunctionUtils.setFunctionRuntimeContext(timeoutFunction, getRuntimeContext()); - FunctionUtils.openFunction(timeoutFunction, parameters); + FunctionUtils.openFunction(timeoutFunction, openContext); } @Override diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java index 7aab6000931da..e5214def3871e 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/nfa/NFA.java @@ -19,6 +19,7 @@ package org.apache.flink.cep.nfa; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; @@ -201,7 +202,7 @@ public void open(RuntimeContext cepRuntimeContext, Configuration conf) throws Ex for (StateTransition transition : state.getStateTransitions()) { IterativeCondition condition = transition.getCondition(); FunctionUtils.setFunctionRuntimeContext(condition, cepRuntimeContext); - FunctionUtils.openFunction(condition, conf); + FunctionUtils.openFunction(condition, DefaultOpenContext.INSTANCE); } } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichCompositeIterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichCompositeIterativeCondition.java index 5b1aea52b719b..5e8416ee3b9e0 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichCompositeIterativeCondition.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichCompositeIterativeCondition.java @@ -18,9 +18,9 @@ package org.apache.flink.cep.pattern.conditions; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Preconditions; /** @@ -58,10 +58,10 @@ public void setRuntimeContext(RuntimeContext t) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); for (IterativeCondition nestedCondition : nestedConditions) { - FunctionUtils.openFunction(nestedCondition, parameters); + FunctionUtils.openFunction(nestedCondition, openContext); } } diff --git a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichIterativeCondition.java b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichIterativeCondition.java index 549e98468ea0b..4aaf582ecc5db 100644 --- a/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichIterativeCondition.java +++ b/flink-libraries/flink-cep/src/main/java/org/apache/flink/cep/pattern/conditions/RichIterativeCondition.java @@ -19,6 +19,7 @@ package org.apache.flink.cep.pattern.conditions; import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; @@ -27,8 +28,7 @@ /** * Rich variant of the {@link IterativeCondition}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and teardown - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. */ public abstract class RichIterativeCondition extends IterativeCondition implements RichFunction { @@ -67,7 +67,13 @@ public IterationRuntimeContext getIterationRuntimeContext() { // -------------------------------------------------------------------------------------------- @Override - public void open(Configuration parameters) throws Exception {} + public void open(OpenContext openContext) throws Exception {} + + @Override + public void open(Configuration parameters) throws Exception { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } @Override public void close() throws Exception {} diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java index da96b9416a94b..216de5f0433f4 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/CEPITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.eventtime.TimestampAssignerSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.common.typeutils.base.LongSerializer; @@ -867,7 +868,7 @@ public boolean filter(Event value, Context ctx) new RichPatternFlatSelectFunction() { @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { try { getRuntimeContext() .getMapState( @@ -963,7 +964,7 @@ public boolean filter(Event value, Context ctx) .select( new RichPatternSelectFunction() { @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { try { getRuntimeContext() .getMapState( diff --git a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepRuntimeContextTest.java b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepRuntimeContextTest.java index 07fbe2be4dd13..1b106149a2837 100644 --- a/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepRuntimeContextTest.java +++ b/flink-libraries/flink-cep/src/test/java/org/apache/flink/cep/operator/CepRuntimeContextTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.cache.DistributedCache; import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.AggregatingStateDescriptor; @@ -285,8 +286,8 @@ private static class VerifyRuntimeContextProcessFunction boolean processMatchCalled = false; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); verifyContext(); openCalled = true; } diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateBootstrapFunction.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateBootstrapFunction.java index e0b084f06c918..d2890e8828857 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateBootstrapFunction.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateBootstrapFunction.java @@ -20,7 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.AbstractRichFunction; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.TimerService; /** @@ -33,7 +33,7 @@ * org.apache.flink.api.common.functions.RichFunction}. Therefore, access to the {@link * org.apache.flink.api.common.functions.RuntimeContext} is always available and setup and teardown * methods can be implemented. See {@link - * org.apache.flink.api.common.functions.RichFunction#open(Configuration)} and {@link + * org.apache.flink.api.common.functions.RichFunction#open(OpenContext)} and {@link * org.apache.flink.api.common.functions.RichFunction#close()}. * * @param Type of the keys. diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateReaderFunction.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateReaderFunction.java index bf94ec5adef0b..5bbf8c981fafa 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateReaderFunction.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/functions/KeyedStateReaderFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; @@ -39,7 +40,7 @@ * org.apache.flink.api.common.functions.RichFunction}. Therefore, access to the {@link * org.apache.flink.api.common.functions.RuntimeContext} is always available and setup and teardown * methods can be implemented. See {@link - * org.apache.flink.api.common.functions.RichFunction#open(Configuration)} and {@link + * org.apache.flink.api.common.functions.RichFunction#open(OpenContext)} and {@link * org.apache.flink.api.common.functions.RichFunction#close()}. * * @param Type of the keys diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/StateReaderOperator.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/StateReaderOperator.java index f29fc9aa5c3bf..67c24ee0940ec 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/StateReaderOperator.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/StateReaderOperator.java @@ -20,12 +20,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.KeyedStateBackend; import org.apache.flink.state.api.runtime.SavepointRuntimeContext; import org.apache.flink.state.api.runtime.VoidTriggerable; @@ -103,7 +103,7 @@ protected final InternalTimerService getInternalTimerService(String name) { } public void open() throws Exception { - FunctionUtils.openFunction(function, new Configuration()); + FunctionUtils.openFunction(function, DefaultOpenContext.INSTANCE); } public void close() throws Exception { diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/window/EvictingWindowReaderFunction.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/window/EvictingWindowReaderFunction.java index cb4f2ae28ded3..3039135429ff8 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/window/EvictingWindowReaderFunction.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/input/operator/window/EvictingWindowReaderFunction.java @@ -19,9 +19,9 @@ package org.apache.flink.state.api.input.operator.window; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.state.api.functions.WindowReaderFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -49,8 +49,8 @@ protected EvictingWindowReaderFunction(WindowReaderFunction wrap } @Override - public void open(Configuration parameters) throws Exception { - FunctionUtils.openFunction(wrappedFunction, parameters); + public void open(OpenContext openContext) throws Exception { + FunctionUtils.openFunction(wrappedFunction, openContext); } @Override diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedOneInputStreamTaskRunner.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedOneInputStreamTaskRunner.java index 4c2426f8b34ee..5ce1ebba7c85f 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedOneInputStreamTaskRunner.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/BoundedOneInputStreamTaskRunner.java @@ -19,8 +19,8 @@ package org.apache.flink.state.api.output; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapPartitionFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.state.api.functions.Timestamper; import org.apache.flink.state.api.runtime.SavepointEnvironment; @@ -67,8 +67,8 @@ public BoundedOneInputStreamTaskRunner( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); env = new SavepointEnvironment.Builder(getRuntimeContext(), maxParallelism) diff --git a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/OperatorSubtaskStateReducer.java b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/OperatorSubtaskStateReducer.java index 9c6de1cb1cff7..24aecb3acb471 100644 --- a/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/OperatorSubtaskStateReducer.java +++ b/flink-libraries/flink-state-processing-api/src/main/java/org/apache/flink/state/api/output/OperatorSubtaskStateReducer.java @@ -19,8 +19,8 @@ package org.apache.flink.state.api.output; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.OperatorState; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.jobgraph.OperatorID; @@ -54,8 +54,8 @@ public OperatorSubtaskStateReducer(OperatorID operatorID, int maxParallelism) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); } @Override diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderITTestBase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderITTestBase.java index 69b485f4aaabf..a93c46bed3ecd 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderITTestBase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderITTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.state.api; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.MapStateDescriptor; @@ -27,7 +28,6 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.state.FunctionInitializationContext; @@ -277,7 +277,7 @@ private StatefulOperator( } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { elements = new ArrayList<>(); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderKeyedStateITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderKeyedStateITCase.java index c313bfd4c2a6e..7550c4b07a253 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderKeyedStateITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/DataSetSavepointReaderKeyedStateITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.state.api; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; @@ -85,7 +86,7 @@ private static class KeyedStatefulOperator extends KeyedProcessFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(valueState); } @@ -104,10 +105,16 @@ private static class Reader extends KeyedStateReaderFunction { private transient ValueState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(valueState); } + @Override + public void open(Configuration parameters) { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey(Integer key, Context ctx, Collector out) throws Exception { Pojo pojo = new Pojo(); diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointDeepCopyTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointDeepCopyTest.java index baf5950d5eb8f..20472c43fbc7c 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointDeepCopyTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointDeepCopyTest.java @@ -18,6 +18,7 @@ package org.apache.flink.state.api; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; @@ -83,7 +84,7 @@ static class WordMapBootstrapper extends KeyedStateBootstrapFunction> state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor> descriptor = new ValueStateDescriptor<>("state", Types.TUPLE(Types.STRING, Types.STRING)); state = getRuntimeContext().getState(descriptor); @@ -103,12 +104,18 @@ static class ReadFunction extends KeyedStateReaderFunction> state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor> stateDescriptor = new ValueStateDescriptor<>("state", Types.TUPLE(Types.STRING, Types.STRING)); state = getRuntimeContext().getState(stateDescriptor); } + @Override + public void open(Configuration parameters) throws Exception { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey(String key, Context ctx, Collector> out) throws Exception { diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java index 4ed0da720c716..ff0596eafc0ee 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderITTestBase.java @@ -19,13 +19,13 @@ package org.apache.flink.state.api; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.time.Deadline; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.SavepointFormatType; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.runtime.state.FunctionInitializationContext; @@ -276,7 +276,7 @@ private StatefulOperator( } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { elements = new ArrayList<>(); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java index 338eb659d9634..c4b0e4385f58c 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointReaderKeyedStateITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.state.api; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; @@ -86,7 +87,7 @@ private static class KeyedStatefulOperator extends KeyedProcessFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(valueState); } @@ -105,10 +106,16 @@ private static class Reader extends KeyedStateReaderFunction { private transient ValueState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(valueState); } + @Override + public void open(Configuration parameters) { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey(Integer key, Context ctx, Collector out) throws Exception { Pojo pojo = new Pojo(); diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java index 3669817a6aaf5..1ea18454a6dfd 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.state.api; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; @@ -27,7 +28,6 @@ import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.state.FunctionInitializationContext; @@ -277,7 +277,7 @@ public static class AccountBootstrapper extends KeyedStateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total", Types.DOUBLE); state = getRuntimeContext().getState(descriptor); @@ -294,8 +294,8 @@ public static class UpdateAndGetAccount extends RichFlatMapFunction state; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total", Types.DOUBLE); @@ -321,7 +321,7 @@ public static class ModifyProcessFunction extends StateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { numbers = new ArrayList<>(); } @@ -351,7 +351,7 @@ public static class StatefulOperator extends RichMapFunction ListState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { numbers = new ArrayList<>(); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterUidModificationITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterUidModificationITCase.java index 9aa77ea52db07..e6c83ec7c30e4 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterUidModificationITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/SavepointWriterUidModificationITCase.java @@ -19,12 +19,12 @@ package org.apache.flink.state.api; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.state.api.functions.KeyedStateBootstrapFunction; @@ -224,7 +224,7 @@ public static class StateBootstrapper extends KeyedStateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(STATE_DESCRIPTOR); } @@ -239,7 +239,7 @@ public static class StateReader extends RichMapFunction { private transient ValueState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(STATE_DESCRIPTOR); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/WritableSavepointITCase.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/WritableSavepointITCase.java index f464f445e081d..1d386ef084d3e 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/WritableSavepointITCase.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/WritableSavepointITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.state.api; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; @@ -28,7 +29,6 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings; @@ -274,7 +274,7 @@ public static class AccountBootstrapper extends KeyedStateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total", Types.DOUBLE); state = getRuntimeContext().getState(descriptor); @@ -291,8 +291,8 @@ public static class UpdateAndGetAccount extends RichFlatMapFunction state; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ValueStateDescriptor descriptor = new ValueStateDescriptor<>("total", Types.DOUBLE); @@ -318,7 +318,7 @@ public static class ModifyProcessFunction extends StateBootstrapFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { numbers = new ArrayList<>(); } @@ -348,7 +348,7 @@ public static class StatefulOperator extends RichMapFunction ListState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { numbers = new ArrayList<>(); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java index 550cfce158346..d86bc619c2339 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/input/KeyedStateInputFormatTest.java @@ -18,6 +18,7 @@ package org.apache.flink.state.api.input; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; @@ -249,10 +250,16 @@ static class ReaderFunction extends KeyedStateReaderFunction { ValueState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(stateDescriptor); } + @Override + public void open(Configuration parameters) throws Exception { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey( Integer key, KeyedStateReaderFunction.Context ctx, Collector out) @@ -265,10 +272,16 @@ static class DoubleReaderFunction extends KeyedStateReaderFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(stateDescriptor); } + @Override + public void open(Configuration parameters) throws Exception { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey( Integer key, KeyedStateReaderFunction.Context ctx, Collector out) @@ -281,10 +294,16 @@ public void readKey( static class InvalidReaderFunction extends KeyedStateReaderFunction { @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { getRuntimeContext().getState(stateDescriptor); } + @Override + public void open(Configuration parameters) throws Exception { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey( Integer key, KeyedStateReaderFunction.Context ctx, Collector out) @@ -298,7 +317,7 @@ static class StatefulFunction extends RichFlatMapFunction { ValueState state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(stateDescriptor); } @@ -312,7 +331,7 @@ static class StatefulFunctionWithTime extends KeyedProcessFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(stateDescriptor); } @@ -329,10 +348,16 @@ static class TimerReaderFunction extends KeyedStateReaderFunction state; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { state = getRuntimeContext().getState(stateDescriptor); } + @Override + public void open(Configuration parameters) throws Exception { + throw new UnsupportedOperationException( + "This method is deprecated and shouldn't be invoked. Please use open(OpenContext) instead."); + } + @Override public void readKey( Integer key, KeyedStateReaderFunction.Context ctx, Collector out) diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/KeyedStateBootstrapOperatorTest.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/KeyedStateBootstrapOperatorTest.java index 39af5bb8d453b..aeab132053b4d 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/KeyedStateBootstrapOperatorTest.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/output/KeyedStateBootstrapOperatorTest.java @@ -18,12 +18,12 @@ package org.apache.flink.state.api.output; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.RocksDBStateBackend; import org.apache.flink.core.fs.Path; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -185,7 +185,7 @@ private static class SimpleBootstrapFunction extends KeyedStateBootstrapFunction private ValueState state; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { state = getRuntimeContext().getState(descriptor); } @@ -200,7 +200,7 @@ private static class StreamingFunction extends RichMapFunction { private ValueState state; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { state = getRuntimeContext().getState(descriptor); } diff --git a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/utils/WaitingSource.java b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/utils/WaitingSource.java index 92145b77ded09..6cc5f7f339420 100644 --- a/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/utils/WaitingSource.java +++ b/flink-libraries/flink-state-processing-api/src/test/java/org/apache/flink/state/api/utils/WaitingSource.java @@ -18,10 +18,11 @@ package org.apache.flink.state.api.utils; +import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.functions.source.SourceFunction; @@ -54,9 +55,9 @@ public void setRuntimeContext(RuntimeContext t) { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { if (source instanceof RichSourceFunction) { - ((RichSourceFunction) source).open(parameters); + ((RichSourceFunction) source).open(DefaultOpenContext.INSTANCE); } } diff --git a/flink-python/src/main/java/org/apache/flink/python/util/PythonConnectorUtils.java b/flink-python/src/main/java/org/apache/flink/python/util/PythonConnectorUtils.java index 987c8acf7c532..a187e4332c89a 100644 --- a/flink-python/src/main/java/org/apache/flink/python/util/PythonConnectorUtils.java +++ b/flink-python/src/main/java/org/apache/flink/python/util/PythonConnectorUtils.java @@ -17,8 +17,8 @@ package org.apache.flink.python.util; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.conversion.RowRowConverter; @@ -113,8 +113,8 @@ public RowRowMapper(DataType dataType) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); converter = RowRowConverter.create(dataType); converter.open(getRuntimeContext().getUserCodeClassLoader()); } diff --git a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/sources/ArrowSourceFunction.java b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/sources/ArrowSourceFunction.java index abd3de92296ed..17e7f5fcd681c 100644 --- a/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/sources/ArrowSourceFunction.java +++ b/flink-python/src/main/java/org/apache/flink/table/runtime/arrow/sources/ArrowSourceFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.arrow.sources; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -27,7 +28,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.api.java.typeutils.runtime.TupleSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -109,7 +109,7 @@ public class ArrowSourceFunction extends RichParallelSourceFunction } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { allocator = ArrowUtils.getRootAllocator() .newChildAllocator("ArrowSourceFunction", 0, Long.MAX_VALUE); diff --git a/flink-python/src/test/java/org/apache/flink/python/util/PartitionCustomTestMapFunction.java b/flink-python/src/test/java/org/apache/flink/python/util/PartitionCustomTestMapFunction.java index 7555d6faf58ec..db7634328339d 100644 --- a/flink-python/src/test/java/org/apache/flink/python/util/PartitionCustomTestMapFunction.java +++ b/flink-python/src/test/java/org/apache/flink/python/util/PartitionCustomTestMapFunction.java @@ -18,8 +18,8 @@ package org.apache.flink.python.util; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.types.Row; /** @@ -31,7 +31,7 @@ public class PartitionCustomTestMapFunction extends RichMapFunction { private int currentTaskIndex; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { this.currentTaskIndex = getRuntimeContext().getIndexOfThisSubtask(); } diff --git a/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java b/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java index 125c1317352c9..48fe8b96ec114 100644 --- a/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java +++ b/flink-python/src/test/java/org/apache/flink/table/utils/TestCollectionTableFactory.java @@ -19,11 +19,11 @@ package org.apache.flink.table.utils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.io.CollectionInputFormat; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.datastream.DataStreamSource; @@ -309,7 +309,7 @@ public UnsafeMemorySinkFunction( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { serializer = outputType.createSerializer(new ExecutionConfig()); } diff --git a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java index 7d6742c2e99f8..d238bf492e392 100644 --- a/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java +++ b/flink-queryable-state/flink-queryable-state-runtime/src/test/java/org/apache/flink/queryablestate/itcases/AbstractQueryableStateTestBase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.AggregatingState; @@ -43,7 +44,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.configuration.Configuration; import org.apache.flink.queryablestate.client.QueryableStateClient; import org.apache.flink.queryablestate.client.VoidNamespace; import org.apache.flink.queryablestate.client.VoidNamespaceSerializer; @@ -833,8 +833,8 @@ public Integer getKey(Tuple2 value) { private transient MapState> mapState; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); mapState = getRuntimeContext().getMapState(mapStateDescriptor); } @@ -936,8 +936,8 @@ public Integer getKey(Tuple2 value) { private transient ListState listState; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); listState = getRuntimeContext().getListState(listStateDescriptor); } @@ -1100,8 +1100,8 @@ private static class TestAscendingValueSource } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); } @Override @@ -1155,8 +1155,8 @@ private static class TestKeyRangeSource } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (getRuntimeContext().getIndexOfThisSubtask() == 0) { LATEST_CHECKPOINT_ID.set(0L); } diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java index 51933209e32c1..4e87538ae152d 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/BatchTask.java @@ -21,8 +21,10 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.distributions.DataDistribution; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -500,7 +502,7 @@ protected void run() throws Exception { if (this.stub != null) { try { Configuration stubConfig = this.config.getStubParameters(); - FunctionUtils.openFunction(this.stub, stubConfig); + FunctionUtils.openFunction(this.stub, DefaultOpenContext.INSTANCE); stubOpen = true; } catch (Throwable t) { throw new Exception( @@ -1481,7 +1483,7 @@ public static Collector initOutputs( /** * Opens the given stub using its {@link - * org.apache.flink.api.common.functions.RichFunction#open(Configuration)} method. If the open + * org.apache.flink.api.common.functions.RichFunction#open(OpenContext)} method. If the open * call produces an exception, a new exception with a standard error message is created, using * the encountered exception as its cause. * @@ -1491,7 +1493,7 @@ public static Collector initOutputs( */ public static void openUserCode(Function stub, Configuration parameters) throws Exception { try { - FunctionUtils.openFunction(stub, parameters); + FunctionUtils.openFunction(stub, DefaultOpenContext.INSTANCE); } catch (Throwable t) { throw new Exception( "The user defined 'open(Configuration)' method in " diff --git a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningSpillingBehaviour.java b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningSpillingBehaviour.java index d2f90643a7168..a4d472b0f7dcb 100644 --- a/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningSpillingBehaviour.java +++ b/flink-runtime/src/main/java/org/apache/flink/runtime/operators/sort/CombiningSpillingBehaviour.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators.sort; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -61,7 +62,7 @@ final class CombiningSpillingBehaviour implements SpillingThread.SpillingBeha @Override public void open() { try { - FunctionUtils.openFunction(combineFunction, udfConfig); + FunctionUtils.openFunction(combineFunction, DefaultOpenContext.INSTANCE); } catch (Throwable t) { throw new FlinkRuntimeException( "The user-defined combiner failed in its 'open()' method.", t); diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningExternalSorterITCase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningExternalSorterITCase.java index 89c62413cb277..405d818b6aa40 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningExternalSorterITCase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/CombiningExternalSorterITCase.java @@ -19,13 +19,13 @@ package org.apache.flink.runtime.operators.sort; import org.apache.flink.api.common.functions.GroupCombineFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerFactory; import org.apache.flink.api.common.typeutils.base.IntComparator; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable; @@ -374,7 +374,7 @@ public void reduce( Collector> out) {} @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened = true; } @@ -413,7 +413,7 @@ public void reduce( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { opened = true; } diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSorterTest.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSorterTest.java index f8861f83a4a55..42edcbc118435 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSorterTest.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/sort/ExternalSorterTest.java @@ -18,6 +18,7 @@ package org.apache.flink.runtime.operators.sort; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigOption; @@ -144,10 +145,9 @@ private static final class RichCombiner boolean isClosed = false; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { assertThat(isOpen).withFailMessage("UDF was already opened").isFalse(); isOpen = true; - assertThat(parameters.get(testOption)).isEqualTo("TEST"); } @Override diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java index fdc772836247f..ea4b0a34188e2 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/BinaryOperatorTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.testutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -210,7 +211,7 @@ protected void testDriverInternal(Driver driver, Class stubClass) throws Excepti // open stub implementation try { - FunctionUtils.openFunction(this.stub, getTaskConfig().getStubParameters()); + FunctionUtils.openFunction(this.stub, DefaultOpenContext.INSTANCE); stubOpen = true; } catch (Throwable t) { throw new Exception( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java index 09909ec6883aa..dd08a08eeaf6d 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/DriverTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.testutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -215,7 +216,7 @@ protected void testDriverInternal(Driver driver, Class stubClass) throws Excepti // open stub implementation try { - FunctionUtils.openFunction(this.stub, getTaskConfig().getStubParameters()); + FunctionUtils.openFunction(this.stub, DefaultOpenContext.INSTANCE); stubOpen = true; } catch (Throwable t) { throw new Exception( diff --git a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java index 8c19fcc532b0d..6c35e6eb2b7af 100644 --- a/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java +++ b/flink-runtime/src/test/java/org/apache/flink/runtime/operators/testutils/UnaryOperatorTestBase.java @@ -19,6 +19,7 @@ package org.apache.flink.runtime.operators.testutils; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.typeutils.TypeComparator; @@ -216,7 +217,7 @@ protected void testDriverInternal(Driver driver, Class stubClass) throws Excepti // open stub implementation try { - FunctionUtils.openFunction(this.stub, getTaskConfig().getStubParameters()); + FunctionUtils.openFunction(this.stub, DefaultOpenContext.INSTANCE); stubOpen = true; } catch (Throwable t) { throw new Exception( diff --git a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java index 9f8cb8a08f54c..45809a076db54 100644 --- a/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java +++ b/flink-scala/src/main/java/org/apache/flink/api/scala/operators/ScalaAggregateOperator.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.operators.Keys; import org.apache.flink.api.common.operators.Operator; @@ -37,7 +38,6 @@ import org.apache.flink.api.java.operators.SingleInputOperator; import org.apache.flink.api.java.typeutils.TupleTypeInfoBase; import org.apache.flink.api.java.typeutils.runtime.TupleSerializerBase; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import org.apache.flink.util.Preconditions; @@ -280,7 +280,7 @@ public AggregatingUdf( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { for (AggregationFunction aggFunction : aggFunctions) { aggFunction.initializeAggregate(); } diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksIncrementalCheckpointRescalingTest.java b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksIncrementalCheckpointRescalingTest.java index baf418f5d17dd..bdc85ef2e5d5d 100644 --- a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksIncrementalCheckpointRescalingTest.java +++ b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksIncrementalCheckpointRescalingTest.java @@ -18,11 +18,11 @@ package org.apache.flink.contrib.streaming.state; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateAssignmentOperation; import org.apache.flink.runtime.state.KeyGroupRange; @@ -428,8 +428,8 @@ private class TestKeyedFunction extends KeyedProcessFunction counterState; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); counterState = this.getRuntimeContext() .getState(new ValueStateDescriptor<>("counter", Integer.class)); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java index c5599c69c819c..88373b904a4bb 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/checkpoint/ListCheckpointed.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.api.checkpoint; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.configuration.Configuration; import java.io.Serializable; import java.util.List; @@ -152,7 +152,7 @@ public interface ListCheckpointed { * *

Important: When implementing this interface together with {@link RichFunction}, * then the {@code restoreState()} method is called before {@link - * RichFunction#open(Configuration)}. + * RichFunction#open(OpenContext)}. * * @param state The state to be restored as a list of atomic sub-states. * @throws Exception Throwing an exception in this method causes the recovery to fail. The exact diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java index 5b9c594045412..6ce5ab9be85a2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/KeyedProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.TimeDomain; import org.apache.flink.streaming.api.TimerService; import org.apache.flink.util.Collector; @@ -41,8 +42,8 @@ * org.apache.flink.api.common.functions.RichFunction}. Therefore, access to the {@link * org.apache.flink.api.common.functions.RuntimeContext} is always available and setup and teardown * methods can be implemented. See {@link - * org.apache.flink.api.common.functions.RichFunction#open(org.apache.flink.configuration.Configuration)} - * and {@link org.apache.flink.api.common.functions.RichFunction#close()}. + * org.apache.flink.api.common.functions.RichFunction#open(OpenContext)} and {@link + * org.apache.flink.api.common.functions.RichFunction#close()}. * * @param Type of the key. * @param Type of the input elements. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java index c635e05530042..fc310ab98c01c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/ProcessFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.TimeDomain; import org.apache.flink.streaming.api.TimerService; import org.apache.flink.util.Collector; @@ -41,8 +42,8 @@ * org.apache.flink.api.common.functions.RichFunction}. Therefore, access to the {@link * org.apache.flink.api.common.functions.RuntimeContext} is always available and setup and teardown * methods can be implemented. See {@link - * org.apache.flink.api.common.functions.RichFunction#open(org.apache.flink.configuration.Configuration)} - * and {@link org.apache.flink.api.common.functions.RichFunction#close()}. + * org.apache.flink.api.common.functions.RichFunction#open(OpenContext)} and {@link + * org.apache.flink.api.common.functions.RichFunction#close()}. * * @param Type of the input elements. * @param Type of the output elements. 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 de2bb129b9d85..1cf2565056b53 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 @@ -32,6 +32,7 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.AggregatingState; @@ -55,8 +56,7 @@ /** * Rich variant of the {@link AsyncFunction}. As a {@link RichFunction}, it gives access to the * {@link RuntimeContext} and provides setup and teardown methods: {@link - * RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * *

State related apis in {@link RuntimeContext} are not supported yet because the key may get * changed while accessing states in the working thread. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java index 22a2fc9faac90..dfcd61adb71bf 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/OutputFormatSinkFunction.java @@ -19,6 +19,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.io.CleanupWhenUnsuccessful; import org.apache.flink.api.common.io.OutputFormat; @@ -58,9 +59,9 @@ public OutputFormatSinkFunction(OutputFormat format) { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { RuntimeContext context = getRuntimeContext(); - format.configure(parameters); + format.configure(new Configuration()); int indexInSubtaskGroup = context.getIndexOfThisSubtask(); int currentNumberOfSubtasks = context.getNumberOfParallelSubtasks(); format.open( diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java index c5c34c3ce80cd..e5706d129a562 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/PrintSinkFunction.java @@ -19,8 +19,8 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.PrintSinkOutputWriter; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; /** @@ -67,8 +67,8 @@ public PrintSinkFunction(final String sinkIdentifier, final boolean stdErr) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext(); writer.open(context.getIndexOfThisSubtask(), context.getNumberOfParallelSubtasks()); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java index fd85bc0072d82..930a0bf7cb7b2 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/sink/SocketClientSink.java @@ -18,8 +18,8 @@ package org.apache.flink.streaming.api.functions.sink; import org.apache.flink.annotation.PublicEvolving; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.serialization.SerializationSchema; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.SerializableObject; import org.slf4j.Logger; @@ -130,10 +130,10 @@ public SocketClientSink( /** * Initialize the connection with the Socket in the server. * - * @param parameters Configuration. + * @param openContext the context. */ @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { try { synchronized (lock) { createConnection(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java index 2c32d276f47c5..0e03ca4425106 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/ContinuousFileMonitoringFunction.java @@ -19,6 +19,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.io.FileInputFormat; import org.apache.flink.api.common.io.FilePathFilter; import org.apache.flink.api.common.state.ListState; @@ -199,9 +200,9 @@ public void initializeState(FunctionInitializationContext context) throws Except } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - format.configure(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + format.configure(new Configuration()); if (LOG.isDebugEnabled()) { LOG.debug( diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java index 9226ce3237177..9cb06ea7308c8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/FromSplittableIteratorFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.api.functions.source; import org.apache.flink.annotation.PublicEvolving; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.util.SplittableIterator; import java.util.Iterator; @@ -47,7 +47,7 @@ public FromSplittableIteratorFunction(SplittableIterator iterator) { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { int numberOfSubTasks = getRuntimeContext().getNumberOfParallelSubtasks(); int indexofThisSubTask = getRuntimeContext().getIndexOfThisSubtask(); iterator = fullIterator.split(numberOfSubTasks)[indexofThisSubTask]; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java index 236e193166962..d9387ab0aaf4b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunction.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.functions.source; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -62,13 +63,13 @@ public InputFormatSourceFunction(InputFormat format, TypeInformation idTypeI } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); sessionIds = new ArrayList<>(64); sessionIdsPerSnapshot = new ArrayDeque<>(); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java index 666e0ca48e6ff..8b56c47e026b8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichParallelSourceFunction.java @@ -19,6 +19,7 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; /** * Base class for implementing a parallel data source. Upon execution, the runtime will execute as @@ -26,8 +27,8 @@ * *

The data source has access to context information (such as the number of parallel instances of * the source, and which parallel instance the current instance is) via {@link - * #getRuntimeContext()}. It also provides additional life-cycle methods ({@link - * #open(org.apache.flink.configuration.Configuration)} and {@link #close()}. + * #getRuntimeContext()}. It also provides additional life-cycle methods ({@link #open(OpenContext)} + * and {@link #close()}. * * @param The type of the records produced by this source. * @deprecated This class is based on the {@link diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java index 5f6110137c35f..bd9249c3c8378 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/RichSourceFunction.java @@ -19,11 +19,12 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; /** * Base class for implementing a parallel data source that has access to context information (via - * {@link #getRuntimeContext()}) and additional life-cycle methods ({@link - * #open(org.apache.flink.configuration.Configuration)} and {@link #close()}. + * {@link #getRuntimeContext()}) and additional life-cycle methods ({@link #open(OpenContext)} and + * {@link #close()}. * *

This class is useful when implementing parallel sources where different parallel subtasks need * to perform different work. Typical patterns for that are: diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java index 600570307cae9..ceacbc10be4c3 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/datagen/DataGeneratorSource.java @@ -19,7 +19,7 @@ package org.apache.flink.streaming.api.functions.source.datagen; import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -81,8 +81,8 @@ public DataGeneratorSource( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (numberOfRows != null) { final int stepSize = getRuntimeContext().getNumberOfParallelSubtasks(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java index e4b4d85e911d0..29ee9a620ee17 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessAllWindowFunction.java @@ -19,10 +19,10 @@ package org.apache.flink.streaming.api.functions.windowing; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; @@ -77,8 +77,8 @@ public void clear(final Context context) throws Exception { } @Override - public void open(Configuration configuration) throws Exception { - FunctionUtils.openFunction(this.windowFunction, configuration); + public void open(OpenContext openContext) throws Exception { + FunctionUtils.openFunction(this.windowFunction, openContext); ctx = new InternalProcessApplyAllWindowContext<>(windowFunction); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java index 6c9602ff40078..1b2cebf26c642 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/ReduceApplyProcessWindowFunction.java @@ -19,10 +19,10 @@ package org.apache.flink.streaming.api.functions.windowing; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; @@ -75,8 +75,8 @@ public void clear(final Context context) throws Exception { } @Override - public void open(Configuration configuration) throws Exception { - FunctionUtils.openFunction(this.windowFunction, configuration); + public void open(OpenContext openContext) throws Exception { + FunctionUtils.openFunction(this.windowFunction, openContext); ctx = new InternalProcessApplyWindowContext<>(windowFunction); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java index e95a2ecc3a077..93f47756ea900 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/windowing/RichWindowFunction.java @@ -20,14 +20,14 @@ import org.apache.flink.annotation.Public; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.streaming.api.windowing.windows.Window; /** * Rich variant of the {@link WindowFunction}. As a {@link RichFunction}, it gives access to the * {@link org.apache.flink.api.common.functions.RuntimeContext} and provides setup and tear-down - * methods: {@link RichFunction#open(org.apache.flink.configuration.Configuration)} and {@link - * RichFunction#close()}. + * methods: {@link RichFunction#open(OpenContext)} and {@link RichFunction#close()}. * * @param The type of the input value. * @param The type of the output value. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java index 70c7c805545f1..a8868c2323082 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperator.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.CheckpointListener; @@ -98,7 +99,7 @@ public void initializeState(StateInitializationContext context) throws Exception @Override public void open() throws Exception { super.open(); - FunctionUtils.openFunction(userFunction, new Configuration()); + FunctionUtils.openFunction(userFunction, DefaultOpenContext.INSTANCE); } @Override diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java index 2f7d07f1d5b08..aeecb135e886c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/collect/CollectSinkFunction.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; @@ -27,7 +28,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; @@ -237,7 +237,7 @@ public void snapshotState(FunctionSnapshotContext context) throws Exception { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { Preconditions.checkState( getRuntimeContext().getNumberOfParallelSubtasks() == 1, "The parallelism of CollectSinkFunction must be 1"); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/experimental/CollectSink.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/experimental/CollectSink.java index 57245fb3e7ab7..806e994fb0811 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/experimental/CollectSink.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/experimental/CollectSink.java @@ -18,8 +18,8 @@ package org.apache.flink.streaming.experimental; import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; @@ -74,10 +74,10 @@ public void invoke(IN value, Context context) throws Exception { /** * Initialize the connection with the Socket in the server. * - * @param parameters Configuration. + * @param openContext the context. */ @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { try { client = new Socket(hostIp, port); outputStream = client.getOutputStream(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java index 54c69484bc001..4980d61ec4378 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalAggregateProcessAllWindowFunction.java @@ -20,9 +20,9 @@ import org.apache.flink.api.common.functions.AggregateFunction; import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.operators.translation.WrappingFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; @@ -57,8 +57,8 @@ public InternalAggregateProcessAllWindowFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ProcessAllWindowFunction wrappedFunction = this.wrappedFunction; this.ctx = new InternalProcessAllWindowContext<>(wrappedFunction); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java index fcbc7e2936e65..8bce2dd8caf1c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalIterableProcessAllWindowFunction.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.runtime.operators.windowing.functions; import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.operators.translation.WrappingFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; @@ -44,8 +44,8 @@ public InternalIterableProcessAllWindowFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ProcessAllWindowFunction wrappedFunction = this.wrappedFunction; this.ctx = new InternalProcessAllWindowContext<>(wrappedFunction); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java index 08743ceeb5949..5fb2c4d077135 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/functions/InternalSingleValueProcessAllWindowFunction.java @@ -19,9 +19,9 @@ package org.apache.flink.streaming.runtime.operators.windowing.functions; import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.operators.translation.WrappingFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.windowing.windows.Window; import org.apache.flink.util.Collector; @@ -46,8 +46,8 @@ public InternalSingleValueProcessAllWindowFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ProcessAllWindowFunction wrappedFunction = this.wrappedFunction; this.ctx = new InternalProcessAllWindowContext<>(wrappedFunction); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java index b197153d1de92..22351c9d0b1c5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/PrintSinkFunctionTest.java @@ -17,7 +17,7 @@ package org.apache.flink.streaming.api.functions; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; import org.apache.flink.streaming.api.functions.sink.SinkContextUtil; import org.apache.flink.streaming.util.MockStreamingRuntimeContext; @@ -65,7 +65,7 @@ public void testPrintSinkStdOut() throws Exception { PrintSinkFunction printSink = new PrintSinkFunction<>(); printSink.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 0)); - printSink.open(new Configuration()); + printSink.open(DefaultOpenContext.INSTANCE); printSink.invoke("hello world!", SinkContextUtil.forTimestamp(0)); @@ -78,7 +78,7 @@ public void testPrintSinkStdOut() throws Exception { public void testPrintSinkStdErr() throws Exception { PrintSinkFunction printSink = new PrintSinkFunction<>(true); printSink.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 0)); - printSink.open(new Configuration()); + printSink.open(DefaultOpenContext.INSTANCE); printSink.invoke("hello world!", SinkContextUtil.forTimestamp(0)); @@ -91,7 +91,7 @@ public void testPrintSinkStdErr() throws Exception { public void testPrintSinkWithPrefix() throws Exception { PrintSinkFunction printSink = new PrintSinkFunction<>(); printSink.setRuntimeContext(new MockStreamingRuntimeContext(false, 2, 1)); - printSink.open(new Configuration()); + printSink.open(DefaultOpenContext.INSTANCE); printSink.invoke("hello world!", SinkContextUtil.forTimestamp(0)); @@ -104,7 +104,7 @@ public void testPrintSinkWithPrefix() throws Exception { public void testPrintSinkWithIdentifierAndPrefix() throws Exception { PrintSinkFunction printSink = new PrintSinkFunction<>("mySink", false); printSink.setRuntimeContext(new MockStreamingRuntimeContext(false, 2, 1)); - printSink.open(new Configuration()); + printSink.open(DefaultOpenContext.INSTANCE); printSink.invoke("hello world!", SinkContextUtil.forTimestamp(0)); @@ -117,7 +117,7 @@ public void testPrintSinkWithIdentifierAndPrefix() throws Exception { public void testPrintSinkWithIdentifierButNoPrefix() throws Exception { PrintSinkFunction printSink = new PrintSinkFunction<>("mySink", false); printSink.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 0)); - printSink.open(new Configuration()); + printSink.open(DefaultOpenContext.INSTANCE); printSink.invoke("hello world!", SinkContextUtil.forTimestamp(0)); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java index 28e3e0d0d86cf..a5a52e272fa7e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/sink/SocketClientSinkTest.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.api.functions.sink; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.serialization.SerializationSchema; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.util.NetUtils; import org.apache.flink.util.TestLogger; @@ -78,7 +78,7 @@ public void run() { try { SocketClientSink simpleSink = new SocketClientSink<>(host, port, simpleSchema, 0); - simpleSink.open(new Configuration()); + simpleSink.open(DefaultOpenContext.INSTANCE); simpleSink.invoke(TEST_MESSAGE + '\n', SinkContextUtil.forTimestamp(0)); simpleSink.close(); } catch (Throwable t) { @@ -113,7 +113,7 @@ public void testSinkAutoFlush() throws Exception { final SocketClientSink simpleSink = new SocketClientSink<>(host, port, simpleSchema, 0, true); - simpleSink.open(new Configuration()); + simpleSink.open(DefaultOpenContext.INSTANCE); final AtomicReference error = new AtomicReference(); @@ -175,7 +175,7 @@ public void run() { SocketClientSink simpleSink = new SocketClientSink<>(host, port, simpleSchema, 0, true); - simpleSink.open(new Configuration()); + simpleSink.open(DefaultOpenContext.INSTANCE); // wait socket server to close serverRunner.join(); @@ -243,7 +243,7 @@ public Void call() throws Exception { host, serverSocket[0].getLocalPort(), simpleSchema, -1, true); // Create the connection - sink.open(new Configuration()); + sink.open(DefaultOpenContext.INSTANCE); // Initial payload => this will be received by the server an then the socket will be // closed. diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java index 821249405e805..5d149c5f2b785 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/functions/source/InputFormatSourceFunctionTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -78,7 +79,7 @@ private void testFormatLifecycle(final boolean midCancel) throws Exception { Assert.assertTrue(!format.isInputFormatOpen); Assert.assertTrue(!format.isSplitOpen); - reader.open(new Configuration()); + reader.open(DefaultOpenContext.INSTANCE); Assert.assertTrue(format.isConfigured); TestSourceContext ctx = new TestSourceContext(reader, format, midCancel, cancelAt); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java index 0b2cdd9a8bd50..3d4125bac394a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/AbstractUdfStreamOperatorLifecycleTest.java @@ -18,6 +18,7 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.configuration.Configuration; @@ -111,7 +112,7 @@ public class AbstractUdfStreamOperatorLifecycleTest { private static final String ALL_METHODS_RICH_FUNCTION = "[close[], getIterationRuntimeContext[], getRuntimeContext[]" - + ", open[class org.apache.flink.configuration.Configuration], setRuntimeContext[interface " + + ", open[class org.apache.flink.configuration.Configuration], open[interface org.apache.flink.api.common.functions.OpenContext], setRuntimeContext[interface " + "org.apache.flink.api.common.functions.RuntimeContext]]"; private static final List ACTUAL_ORDER_TRACKING = @@ -234,9 +235,9 @@ public void setRuntimeContext(RuntimeContext t) { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { ACTUAL_ORDER_TRACKING.add("UDF::open"); - super.open(parameters); + super.open(openContext); } @Override diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java index 2c898ffd79fc7..626d5cbafdcd7 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/StreamFilterTest.java @@ -18,8 +18,8 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; @@ -111,8 +111,8 @@ private static class TestOpenCloseFilterFunction extends RichFilterFunction static int counter = 0; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); synchronized (MyAbstractAsyncFunction.class) { if (counter == 0) { @@ -572,7 +572,7 @@ private JobVertex createChainedVertex( private Integer initialValue = null; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { initialValue = 1; } @@ -1412,8 +1412,8 @@ public int getTryCount(Integer item) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); tryCounts = new HashMap<>(); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java index 77369082d2dfb..3f4c2d13f6e10 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamFlatMapTest.java @@ -17,7 +17,7 @@ package org.apache.flink.streaming.api.operators.co; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.functions.co.CoFlatMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoFlatMapFunction; import org.apache.flink.streaming.api.watermark.Watermark; @@ -139,8 +139,8 @@ private static class TestOpenCloseCoFlatMapFunction public static boolean closeCalled = false; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (closeCalled) { Assert.fail("Close called before open."); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java index 8df46a506d07c..825909997e531 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/co/CoStreamMapTest.java @@ -17,7 +17,7 @@ package org.apache.flink.streaming.api.operators.co; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.functions.co.CoMapFunction; import org.apache.flink.streaming.api.functions.co.RichCoMapFunction; import org.apache.flink.streaming.api.watermark.Watermark; @@ -133,8 +133,8 @@ private static class TestOpenCloseCoMapFunction public static boolean closeCalled = false; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (closeCalled) { Assert.fail("Close called before open."); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/collect/utils/CollectSinkFunctionTestWrapper.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/collect/utils/CollectSinkFunctionTestWrapper.java index f95120b0ffbab..5e193ee70bcbe 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/collect/utils/CollectSinkFunctionTestWrapper.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/collect/utils/CollectSinkFunctionTestWrapper.java @@ -20,10 +20,10 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.SerializedListAccumulator; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.io.disk.iomanager.IOManager; import org.apache.flink.runtime.io.disk.iomanager.IOManagerAsync; import org.apache.flink.runtime.memory.MemoryManager; @@ -99,7 +99,7 @@ public void openFunction() throws Exception { function = new CollectSinkFunction<>(serializer, maxBytesPerBatch, ACCUMULATOR_NAME); function.setRuntimeContext(runtimeContext); function.setOperatorEventGateway(gateway); - function.open(new Configuration()); + function.open(DefaultOpenContext.INSTANCE); coordinator.handleEventFromOperator(0, 0, gateway.getNextEvent()); } @@ -109,7 +109,7 @@ public void openFunctionWithState() throws Exception { function.setRuntimeContext(runtimeContext); function.setOperatorEventGateway(gateway); function.initializeState(functionInitializationContext); - function.open(new Configuration()); + function.open(DefaultOpenContext.INSTANCE); coordinator.handleEventFromOperator(0, 0, gateway.getNextEvent()); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java index 37a7265c1a761..87466314d6fbe 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/windowing/functions/InternalWindowFunctionTest.java @@ -20,10 +20,11 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.AggregateFunction; +import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.windowing.ProcessAllWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; import org.apache.flink.streaming.api.functions.windowing.RichAllWindowFunction; @@ -87,10 +88,10 @@ public void testInternalIterableAllWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -131,10 +132,10 @@ public void testInternalIterableProcessAllWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -174,10 +175,10 @@ public void testInternalIterableWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -217,10 +218,10 @@ public void testInternalIterableProcessWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -282,10 +283,10 @@ public void testInternalSingleValueWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -330,10 +331,10 @@ public void testInternalSingleValueAllWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -377,10 +378,10 @@ public void testInternalSingleValueProcessAllWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -423,10 +424,10 @@ public void testInternalSingleValueProcessWindowFunction() throws Exception { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -523,10 +524,10 @@ public Set merge(Set a, Set b) { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); @@ -633,10 +634,10 @@ public Set merge(Set a, Set b) { verify(mock).setOutputType(stringType, execConf); // check open - Configuration config = new Configuration(); + OpenContext openContext = DefaultOpenContext.INSTANCE; - windowFunction.open(config); - verify(mock).open(config); + windowFunction.open(openContext); + verify(mock).open(openContext); // check setRuntimeContext RuntimeContext rCtx = mock(RuntimeContext.class); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java index 7ce4511abe6dd..b3a2434a591a1 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/EvictingWindowOperatorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.operators.windowing; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; @@ -27,7 +28,6 @@ import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ReduceApplyWindowFunction; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; @@ -870,8 +870,8 @@ public RichSumReducer(AtomicInteger closeCalled) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); openCalled = true; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java index fbf8437551f32..83b63be058370 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorMigrationTest.java @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; @@ -31,7 +32,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; @@ -1187,8 +1187,8 @@ private static class RichSumReducer private boolean openCalled = false; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); openCalled = true; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java index bc5ef7f0b05b1..7f71899545be5 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperatorTest.java @@ -19,6 +19,7 @@ package org.apache.flink.streaming.runtime.operators.windowing; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ReducingStateDescriptor; @@ -28,7 +29,6 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.functions.windowing.PassThroughWindowFunction; import org.apache.flink.streaming.api.functions.windowing.ProcessWindowFunction; @@ -3109,8 +3109,8 @@ private static class RichSumReducer private boolean openCalled = false; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); openCalled = true; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java index 936fa21ca9d59..11634de68fd12 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/OneInputStreamTaskTest.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -1236,8 +1237,8 @@ private static class TestOpenCloseMapFunction extends RichMapFunction { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (closeCalled) { fail("Close called before open."); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 585cd51ee1701..bf847b0c6b70e 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback; import org.apache.flink.api.common.state.CheckpointListener; @@ -2729,7 +2730,7 @@ public FailedSource() { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { running = true; } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java index 5272abc7f6c79..13e6c94f2336d 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/TwoInputStreamTaskTest.java @@ -20,10 +20,10 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.base.IntSerializer; import org.apache.flink.api.common.typeutils.base.StringSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Metric; @@ -961,8 +961,8 @@ private static class TestOpenCloseMapFunction } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (closeCalled) { Assert.fail("Close called before open."); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java index d39ce1001a73f..77884aff4d3bc 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/util/SourceFunctionUtil.java @@ -18,9 +18,9 @@ package org.apache.flink.streaming.util; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RichFunction; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.operators.testutils.MockEnvironment; import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; @@ -64,7 +64,7 @@ private static List runRichSourceFunction( RuntimeContext runtimeContext = new StreamingRuntimeContext(operator, environment, new HashMap<>()); ((RichFunction) sourceFunction).setRuntimeContext(runtimeContext); - ((RichFunction) sourceFunction).open(new Configuration()); + ((RichFunction) sourceFunction).open(DefaultOpenContext.INSTANCE); return runNonRichSourceFunction(sourceFunction); } diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala index 24c7a7db978d2..5b4484f66832f 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/async/ScalaRichAsyncFunctionWrapper.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.api.scala.async -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{OpenContext, RuntimeContext} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.async.{ResultFuture => JResultFuture, RichAsyncFunction => JRichAsyncFunction} @@ -46,8 +46,8 @@ final class ScalaRichAsyncFunctionWrapper[IN, OUT](func: RichAsyncFunction[IN, O func.timeout(input, new JavaResultFutureWrapper[OUT](resultFuture)) } - override def open(parameters: Configuration): Unit = { - func.open(parameters) + override def open(openContext: OpenContext): Unit = { + func.open(openContext) } override def close(): Unit = { diff --git a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala index 611391121cbbe..8f69857b0b0a1 100644 --- a/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala +++ b/flink-streaming-scala/src/main/scala/org/apache/flink/streaming/api/scala/function/util/ScalaProcessWindowFunctionWrapper.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.api.scala.function.util -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{OpenContext, RuntimeContext} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.windowing.{ProcessAllWindowFunction => JProcessAllWindowFunction} import org.apache.flink.streaming.api.functions.windowing.{ProcessWindowFunction => JProcessWindowFunction} @@ -94,10 +94,10 @@ final class ScalaProcessWindowFunctionWrapper[IN, OUT, KEY, W <: Window]( } } - override def open(parameters: Configuration): Unit = { - super.open(parameters) + override def open(openContext: OpenContext): Unit = { + super.open(openContext) func match { - case rfunc: ScalaProcessWindowFunction[IN, OUT, KEY, W] => rfunc.open(parameters) + case rfunc: ScalaProcessWindowFunction[IN, OUT, KEY, W] => rfunc.open(openContext) case _ => } } @@ -160,10 +160,10 @@ final class ScalaProcessAllWindowFunctionWrapper[IN, OUT, W <: Window]( } } - override def open(parameters: Configuration): Unit = { - super.open(parameters) + override def open(openContext: OpenContext): Unit = { + super.open(openContext) func match { - case rfunc: ScalaProcessAllWindowFunction[IN, OUT, W] => rfunc.open(parameters) + case rfunc: ScalaProcessAllWindowFunction[IN, OUT, W] => rfunc.open(openContext) case _ => } } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala index edd53b660d746..caa25f9ce0520 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/AsyncDataStreamITCase.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.streaming.api.scala +import org.apache.flink.api.common.functions.OpenContext import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.functions.sink.SinkFunction import org.apache.flink.streaming.api.scala.AsyncDataStreamITCase._ @@ -250,7 +251,7 @@ class AsyncDataStreamITCase(ordered: Boolean) extends AbstractTestBase { class AsyncFunctionWithTimeoutExpired extends RichAsyncFunction[Int, Int] { @transient var invokeLatch: CountDownLatch = _ - override def open(parameters: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { invokeLatch = new CountDownLatch(1) } @@ -275,7 +276,7 @@ class AsyncFunctionWithTimeoutExpired extends RichAsyncFunction[Int, Int] { class AsyncFunctionWithoutTimeoutExpired extends RichAsyncFunction[Int, Int] { @transient var timeoutLatch: CountDownLatch = _ - override def open(parameters: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { timeoutLatch = new CountDownLatch(1) } @@ -296,7 +297,7 @@ class AsyncFunctionWithoutTimeoutExpired extends RichAsyncFunction[Int, Int] { class MyRichAsyncFunction extends RichAsyncFunction[Int, Int] { - override def open(parameters: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { assertEquals(getRuntimeContext.getNumberOfParallelSubtasks, 1) } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichAllWindowFunction.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichAllWindowFunction.scala index 5e67e02f54497..6788d7e208a03 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichAllWindowFunction.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichAllWindowFunction.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.api.scala.testutils -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{OpenContext, RuntimeContext} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.scala.function.RichAllWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window @@ -31,8 +31,8 @@ class CheckingIdentityRichAllWindowFunction[T, W <: Window] extends RichAllWindo } } - override def open(conf: Configuration): Unit = { - super.open(conf) + override def open(openContext: OpenContext): Unit = { + super.open(openContext) CheckingIdentityRichAllWindowFunction.openCalled = true } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessAllWindowFunction.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessAllWindowFunction.scala index ebe9c74b301f2..6068415efde52 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessAllWindowFunction.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessAllWindowFunction.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.api.scala.testutils -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{OpenContext, RuntimeContext} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.scala.function.ProcessAllWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window @@ -32,8 +32,8 @@ class CheckingIdentityRichProcessAllWindowFunction[T, W <: Window] } } - override def open(conf: Configuration): Unit = { - super.open(conf) + override def open(openContext: OpenContext): Unit = { + super.open(openContext) CheckingIdentityRichProcessAllWindowFunction.openCalled = true } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessWindowFunction.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessWindowFunction.scala index 4a22ea940bc1f..6591a1f063cd6 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessWindowFunction.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichProcessWindowFunction.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.api.scala.testutils -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{OpenContext, RuntimeContext} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.scala.function.ProcessWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window @@ -32,8 +32,8 @@ class CheckingIdentityRichProcessWindowFunction[T, K, W <: Window] } } - override def open(conf: Configuration): Unit = { - super.open(conf) + override def open(openContext: OpenContext): Unit = { + super.open(openContext) CheckingIdentityRichProcessWindowFunction.openCalled = true } diff --git a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichWindowFunction.scala b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichWindowFunction.scala index 138bf3d5a04b4..34a8ba71d5ce1 100644 --- a/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichWindowFunction.scala +++ b/flink-streaming-scala/src/test/scala/org/apache/flink/streaming/api/scala/testutils/CheckingIdentityRichWindowFunction.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.streaming.api.scala.testutils -import org.apache.flink.api.common.functions.RuntimeContext +import org.apache.flink.api.common.functions.{OpenContext, RuntimeContext} import org.apache.flink.configuration.Configuration import org.apache.flink.streaming.api.scala.function.RichWindowFunction import org.apache.flink.streaming.api.windowing.windows.Window @@ -31,8 +31,8 @@ class CheckingIdentityRichWindowFunction[T, K, W <: Window] extends RichWindowFu } } - override def open(conf: Configuration): Unit = { - super.open(conf) + override def open(openContext: OpenContext): Unit = { + super.open(openContext) CheckingIdentityRichWindowFunction.openCalled = true } diff --git a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/print/table/PrintTableSinkFactory.java b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/print/table/PrintTableSinkFactory.java index 64c9e185fbcf9..a7ef357d4b84a 100644 --- a/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/print/table/PrintTableSinkFactory.java +++ b/flink-table/flink-table-api-java-bridge/src/main/java/org/apache/flink/connector/print/table/PrintTableSinkFactory.java @@ -19,9 +19,9 @@ package org.apache.flink.connector.print.table; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.PrintSinkOutputWriter; import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; @@ -174,8 +174,8 @@ private RowDataPrintFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); StreamingRuntimeContext context = (StreamingRuntimeContext) getRuntimeContext(); writer.open(context.getIndexOfThisSubtask(), context.getNumberOfParallelSubtasks()); } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala index fca0edbba5ea2..40a3df6405748 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/ExpressionReducer.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.planner.codegen -import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} +import org.apache.flink.api.common.functions.{MapFunction, OpenContext, RichMapFunction} import org.apache.flink.configuration.{Configuration, PipelineOptions, ReadableConfig} import org.apache.flink.table.api.{TableConfig, TableException} import org.apache.flink.table.data.{DecimalData, GenericRowData, TimestampData} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala index 00d598c84e644..66f56e3bd204c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/MatchCodeGenerator.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.planner.codegen -import org.apache.flink.api.common.functions.Function +import org.apache.flink.api.common.functions.{Function, OpenContext} import org.apache.flink.cep.functions.PatternProcessFunction import org.apache.flink.cep.pattern.conditions.{IterativeCondition, RichIterativeCondition} import org.apache.flink.configuration.Configuration @@ -277,7 +277,7 @@ class MatchCodeGenerator( } @Override - public void open(${classOf[Configuration].getCanonicalName} parameters) throws Exception { + public void open(${classOf[OpenContext].getCanonicalName} openContext) throws Exception { ${ctx.reuseOpenCode()} } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala index 515747be61cfc..6273c649180d8 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/utils/PartitionPruner.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.table.planner.plan.utils -import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} +import org.apache.flink.api.common.functions.{DefaultOpenContext, MapFunction, OpenContext, RichMapFunction} import org.apache.flink.api.common.functions.util.ListCollector import org.apache.flink.configuration.Configuration import org.apache.flink.table.api.{TableConfig, TableException} @@ -126,7 +126,7 @@ object PartitionPruner { val collector = new ListCollector[Boolean](results) try { - richMapFunction.open(new Configuration) + richMapFunction.open(DefaultOpenContext.INSTANCE) // do filter against all partitions allPartitions.foreach { partition => diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestSupportsStagingTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestSupportsStagingTableFactory.java index bdc8a19dfb9cd..2dae8b5b4be3a 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestSupportsStagingTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestSupportsStagingTableFactory.java @@ -18,9 +18,9 @@ package org.apache.flink.table.planner.factories; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; @@ -155,8 +155,8 @@ public StagedSinkFunction(String dataDir, boolean sinkFail) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); File parentDir = new File(dataDir); if (parentDir.exists()) { parentDir.delete(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestUpdateDeleteTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestUpdateDeleteTableFactory.java index 74424cc075c3c..6603a7071a8c6 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestUpdateDeleteTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestUpdateDeleteTableFactory.java @@ -18,10 +18,10 @@ package org.apache.flink.table.planner.factories; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; @@ -618,7 +618,7 @@ private static class DeleteDataSinkFunction extends RichSinkFunction { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { data = registeredRowData.get(dataId); newData = new ArrayList<>(); } @@ -888,7 +888,7 @@ public UpdateDataSinkFunction( } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { oldRows = registeredRowData.get(dataId).toArray(new RowData[0]); updatedRows = new ArrayList<>(); allNewRows = new ArrayList<>(); diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java index a81b171895d17..71e05066d09b8 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/runtime/stream/sql/DataStreamJavaITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeHint; @@ -900,7 +901,7 @@ private Table getComplexUnifiedPipeline(StreamExecutionEnvironment env) { ValueState count; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { count = getRuntimeContext() .getState( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala index 0b10a72521cdc..58413a68b7e4b 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/codegen/WatermarkGeneratorCodeGenTest.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.codegen import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier +import org.apache.flink.api.common.functions.{DefaultOpenContext, OpenContext} import org.apache.flink.configuration.Configuration import org.apache.flink.metrics.MetricGroup import org.apache.flink.streaming.util.MockStreamingRuntimeContext @@ -120,7 +121,7 @@ class WatermarkGeneratorCodeGenTest(useDefinedConstructor: Boolean) { // mock open and close invoking generator.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 1)) } - generator.open(new Configuration()) + generator.open(DefaultOpenContext.INSTANCE) val results = data.map(d => generator.currentWatermark(d)) generator.close() val expected = List( diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala index da56cafd8b64a..3fc912ba44d51 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/factories/utils/TestCollectionTableFactory.scala @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.factories.utils import org.apache.flink.api.common.ExecutionConfig +import org.apache.flink.api.common.functions.OpenContext import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.common.typeutils.TypeSerializer import org.apache.flink.api.java.io.CollectionInputFormat @@ -166,7 +167,7 @@ object TestCollectionTableFactory { class UnsafeMemorySinkFunction(outputType: TypeInformation[Row]) extends RichSinkFunction[Row] { private var serializer: TypeSerializer[Row] = _ - override def open(param: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { serializer = outputType.createSerializer(new ExecutionConfig) } diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala index 8e4d4ce5b7209..6d6e7e84949a6 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/PartitionableSinkITCase.scala @@ -17,6 +17,7 @@ */ package org.apache.flink.table.planner.runtime.batch.sql +import org.apache.flink.api.common.functions.OpenContext import org.apache.flink.api.common.typeinfo.BasicTypeInfo.{INT_TYPE_INFO, STRING_TYPE_INFO} import org.apache.flink.api.common.typeinfo.TypeInformation import org.apache.flink.api.java.typeutils.RowTypeInfo @@ -239,7 +240,7 @@ object PartitionableSinkITCase { class UnsafeMemorySinkFunction(outputType: TypeInformation[Row]) extends RichSinkFunction[Row] { private var resultSet: JLinkedList[String] = _ - override def open(param: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { val taskId = getRuntimeContext.getIndexOfThisSubtask resultSet = RESULT_QUEUE.get(taskId) } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/DefaultExpressionEvaluator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/DefaultExpressionEvaluator.java index 418476ff49b23..31261d574ce38 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/DefaultExpressionEvaluator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/functions/DefaultExpressionEvaluator.java @@ -19,8 +19,8 @@ package org.apache.flink.table.runtime.functions; import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RichFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableException; import org.apache.flink.table.functions.FunctionContext; import org.apache.flink.table.functions.SpecializedFunction.ExpressionEvaluator; @@ -59,7 +59,7 @@ public MethodHandle open(FunctionContext context) { expressionSummary); try { instance = generatedClass.newInstance(context.getUserCodeClassLoader()); - instance.open(new Configuration()); + instance.open(DefaultOpenContext.INSTANCE); final MethodHandles.Lookup publicLookup = MethodHandles.publicLookup(); return publicLookup .findVirtual(instance.getClass(), "eval", methodType) diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java index efa1b5986db7f..acc60591b555e 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/GeneratedWatermarkGeneratorSupplier.java @@ -22,7 +22,7 @@ import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkOutput; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.table.data.RowData; import org.apache.flink.table.watermark.WatermarkEmitStrategy; import org.apache.flink.table.watermark.WatermarkParams; @@ -69,7 +69,7 @@ public GeneratedWatermarkGeneratorSupplier( .newInstance(Thread.currentThread().getContextClassLoader()); try { - innerWatermarkGenerator.open(new Configuration()); + innerWatermarkGenerator.open(DefaultOpenContext.INSTANCE); } catch (Exception e) { throw new RuntimeException("Fail to instantiate generated watermark generator.", e); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java index 716413def6c4e..b1c21c559b2b0 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupAggFunction.java @@ -18,10 +18,10 @@ package org.apache.flink.table.runtime.operators.aggregate; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.utils.JoinedRowData; @@ -102,8 +102,8 @@ public GroupAggFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); // instantiate function StateTtlConfig ttlConfig = createTtlConfig(stateRetentionTime); function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupTableAggFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupTableAggFunction.java index e951e2f4432fc..ad26d48b703b5 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupTableAggFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/aggregate/GroupTableAggFunction.java @@ -18,10 +18,10 @@ package org.apache.flink.table.runtime.operators.aggregate; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.dataview.PerKeyStateDataViewStore; @@ -85,8 +85,8 @@ public GroupTableAggFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); // instantiate function StateTtlConfig ttlConfig = createTtlConfig(stateRetentionTime); function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionBase.java index fa6b04ae8deea..b87a02e7d9c48 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionBase.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/DeduplicateFunctionBase.java @@ -18,12 +18,12 @@ package org.apache.flink.table.runtime.operators.deduplicate; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import static org.apache.flink.table.runtime.util.StateConfigUtil.createTtlConfig; @@ -55,8 +55,8 @@ public DeduplicateFunctionBase( } @Override - public void open(Configuration configure) throws Exception { - super.open(configure); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); ValueStateDescriptor stateDesc = new ValueStateDescriptor<>("deduplicate-state", typeInfo); StateTtlConfig ttlConfig = createTtlConfig(stateRetentionTime); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/ProcTimeDeduplicateKeepLastRowFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/ProcTimeDeduplicateKeepLastRowFunction.java index 979b726675d04..cb4a3cd4636ec 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/ProcTimeDeduplicateKeepLastRowFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/deduplicate/ProcTimeDeduplicateKeepLastRowFunction.java @@ -18,7 +18,7 @@ package org.apache.flink.table.runtime.operators.deduplicate; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; import org.apache.flink.table.runtime.generated.RecordEqualiser; @@ -59,8 +59,8 @@ public ProcTimeDeduplicateKeepLastRowFunction( } @Override - public void open(Configuration configure) throws Exception { - super.open(configure); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); equaliser = genRecordEqualiser.newInstance(getRuntimeContext().getUserCodeClassLoader()); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java index abdaa22c38604..1c70027c52006 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/HashJoinOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.table.runtime.operators.join; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.configuration.AlgorithmOptions; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.BoundedMultiInput; import org.apache.flink.streaming.api.operators.InputSelectable; import org.apache.flink.streaming.api.operators.InputSelection; @@ -120,7 +120,7 @@ public void open() throws Exception { this.condition = parameter.condFuncCode.newInstance(cl); condition.setRuntimeContext(getRuntimeContext()); - condition.open(new Configuration()); + condition.open(DefaultOpenContext.INSTANCE); this.table = new BinaryHashTable( diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinFunction.java index aa5bed0932023..96d3cfad559b1 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/SortMergeJoinFunction.java @@ -18,8 +18,8 @@ package org.apache.flink.table.runtime.operators.join; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.groups.OperatorMetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -208,7 +208,7 @@ public void open( keyComparator = genKeyComparator.newInstance(cl); this.condFunc = condFuncCode.newInstance(cl); condFunc.setRuntimeContext(runtimeContext); - condFunc.open(new Configuration()); + condFunc.open(DefaultOpenContext.INSTANCE); projection1 = projectionCode1.newInstance(cl); projection2 = projectionCode2.newInstance(cl); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/IntervalJoinFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/IntervalJoinFunction.java index ac163d6986f27..f293feacbf275 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/IntervalJoinFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/IntervalJoinFunction.java @@ -18,10 +18,10 @@ package org.apache.flink.table.runtime.operators.join.interval; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.binary.NullAwareGetters; import org.apache.flink.table.data.utils.JoinedRowData; @@ -60,12 +60,12 @@ public IntervalJoinFunction( } @Override - public void open(Configuration config) throws Exception { + public void open(OpenContext openContext) throws Exception { this.joinCondition = joinConditionCode.newInstance(getRuntimeContext().getUserCodeClassLoader()); this.joinConditionCode = null; this.joinCondition.setRuntimeContext(getRuntimeContext()); - this.joinCondition.open(config); + this.joinCondition.open(openContext); this.reusedJoinRowData = new JoinedRowData(); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/TimeIntervalJoin.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/TimeIntervalJoin.java index 433cd76c453af..ba0cee825f572 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/TimeIntervalJoin.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/interval/TimeIntervalJoin.java @@ -18,6 +18,8 @@ package org.apache.flink.table.runtime.operators.join.interval; +import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; @@ -26,7 +28,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.ListTypeInfo; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.co.KeyedCoProcessFunction; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.operators.join.FlinkJoinType; @@ -105,9 +106,9 @@ abstract class TimeIntervalJoin extends KeyedCoProcessFunction createFetcherResultFuture(Configuratio TableFunctionResultFuture resultFuture = generatedResultFuture.newInstance(getRuntimeContext().getUserCodeClassLoader()); FunctionUtils.setFunctionRuntimeContext(resultFuture, getRuntimeContext()); - FunctionUtils.openFunction(resultFuture, parameters); + FunctionUtils.openFunction(resultFuture, DefaultOpenContext.INSTANCE); return resultFuture; } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java index 4788dc5a6bd7d..4d6e035b95ec5 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/AsyncLookupJoinWithCalcRunner.java @@ -18,7 +18,9 @@ package org.apache.flink.table.runtime.operators.join.lookup; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.async.AsyncFunction; @@ -63,8 +65,8 @@ public AsyncLookupJoinWithCalcRunner( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); // try to compile the generated ResultFuture, fail fast if the code is corrupt. generatedCalc.compile(getRuntimeContext().getUserCodeClassLoader()); } @@ -77,7 +79,7 @@ public TableFunctionResultFuture createFetcherResultFuture(Configuratio FlatMapFunction calc = generatedCalc.newInstance(getRuntimeContext().getUserCodeClassLoader()); FunctionUtils.setFunctionRuntimeContext(calc, getRuntimeContext()); - FunctionUtils.openFunction(calc, parameters); + FunctionUtils.openFunction(calc, DefaultOpenContext.INSTANCE); return new TemporalTableCalcResultFuture(calc, joinConditionCollector); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/KeyedLookupJoinWrapper.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/KeyedLookupJoinWrapper.java index e6dd89b9737e8..3913a43549873 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/KeyedLookupJoinWrapper.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/KeyedLookupJoinWrapper.java @@ -18,12 +18,13 @@ package org.apache.flink.table.runtime.operators.join.lookup; +import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.ListSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.MemorySegmentFactory; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; @@ -80,10 +81,10 @@ public KeyedLookupJoinWrapper( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); lookupJoinRunner.setRuntimeContext(getRuntimeContext()); - lookupJoinRunner.open(parameters); + lookupJoinRunner.open(DefaultOpenContext.INSTANCE); if (lookupKeyContainsPrimaryKey) { ValueStateDescriptor valueStateDescriptor = diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java index 0585b5bb76041..cb81845289413 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinRunner.java @@ -19,8 +19,8 @@ package org.apache.flink.table.runtime.operators.join.lookup; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; @@ -62,8 +62,8 @@ public LookupJoinRunner( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.fetcher = generatedFetcher.newInstance(getRuntimeContext().getUserCodeClassLoader()); this.collector = generatedCollector.newInstance(getRuntimeContext().getUserCodeClassLoader()); @@ -74,9 +74,9 @@ public void open(Configuration parameters) throws Exception { FunctionUtils.setFunctionRuntimeContext(fetcher, getRuntimeContext()); FunctionUtils.setFunctionRuntimeContext(collector, getRuntimeContext()); FunctionUtils.setFunctionRuntimeContext(preFilterCondition, getRuntimeContext()); - FunctionUtils.openFunction(fetcher, parameters); - FunctionUtils.openFunction(collector, parameters); - FunctionUtils.openFunction(preFilterCondition, parameters); + FunctionUtils.openFunction(fetcher, openContext); + FunctionUtils.openFunction(collector, openContext); + FunctionUtils.openFunction(preFilterCondition, openContext); this.nullRow = new GenericRowData(tableFieldsCount); this.outRow = new JoinedRowData(); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java index 1f6b9d66499ed..1e72494b0fcb7 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/lookup/LookupJoinWithCalcRunner.java @@ -19,8 +19,8 @@ package org.apache.flink.table.runtime.operators.join.lookup; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.collector.ListenableCollector; import org.apache.flink.table.runtime.generated.FilterCondition; @@ -54,11 +54,11 @@ public LookupJoinWithCalcRunner( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.calc = generatedCalc.newInstance(getRuntimeContext().getUserCodeClassLoader()); FunctionUtils.setFunctionRuntimeContext(calc, getRuntimeContext()); - FunctionUtils.openFunction(calc, parameters); + FunctionUtils.openFunction(calc, openContext); this.calcCollector = new CalcCollector(collector); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java index c7dad64663125..b0009bfe03443 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/stream/AbstractStreamingJoinOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.table.runtime.operators.join.stream; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; @@ -92,7 +92,7 @@ public void open() throws Exception { generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader()); this.joinCondition = new JoinConditionWithNullFilters(condition, filterNullKeys, this); this.joinCondition.setRuntimeContext(getRuntimeContext()); - this.joinCondition.open(new Configuration()); + this.joinCondition.open(DefaultOpenContext.INSTANCE); this.collector = new TimestampedCollector<>(output); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java index 3d3b560f7751f..1c0a61242cb6a 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalProcessTimeJoinOperator.java @@ -18,10 +18,10 @@ package org.apache.flink.table.runtime.operators.join.temporal; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.streaming.api.operators.InternalTimer; import org.apache.flink.streaming.api.operators.TimestampedCollector; @@ -78,7 +78,7 @@ public void open() throws Exception { this.joinCondition = generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader()); FunctionUtils.setFunctionRuntimeContext(joinCondition, getRuntimeContext()); - FunctionUtils.openFunction(joinCondition, new Configuration()); + FunctionUtils.openFunction(joinCondition, DefaultOpenContext.INSTANCE); ValueStateDescriptor rightStateDesc = new ValueStateDescriptor<>("right", rightType); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java index 3588dc9409559..940e15c2b4d03 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/temporal/TemporalRowTimeJoinOperator.java @@ -19,12 +19,12 @@ package org.apache.flink.table.runtime.operators.join.temporal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.VoidNamespace; import org.apache.flink.runtime.state.VoidNamespaceSerializer; import org.apache.flink.streaming.api.operators.InternalTimer; @@ -148,7 +148,7 @@ public void open() throws Exception { joinCondition = generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader()); joinCondition.setRuntimeContext(getRuntimeContext()); - joinCondition.open(new Configuration()); + joinCondition.open(DefaultOpenContext.INSTANCE); nextLeftIndex = getRuntimeContext() diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java index 9acfdd813faee..80c133aa20fb8 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/window/WindowJoinOperator.java @@ -18,11 +18,11 @@ package org.apache.flink.table.runtime.operators.join.window; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.metrics.Meter; @@ -148,7 +148,7 @@ public void open() throws Exception { generatedJoinCondition.newInstance(getRuntimeContext().getUserCodeClassLoader()); this.joinCondition = new JoinConditionWithNullFilters(condition, filterNullKeys, this); this.joinCondition.setRuntimeContext(getRuntimeContext()); - this.joinCondition.open(new Configuration()); + this.joinCondition.open(DefaultOpenContext.INSTANCE); // init state ListStateDescriptor leftRecordStateDesc = diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java index 1645f239a0ec5..1f1d1032d6d4e 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/IterativeConditionRunner.java @@ -18,9 +18,9 @@ package org.apache.flink.table.runtime.operators.match; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.cep.pattern.conditions.RichIterativeCondition; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.generated.GeneratedFunction; @@ -40,10 +40,10 @@ public IterativeConditionRunner( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.function = generatedFunction.newInstance(getRuntimeContext().getUserCodeClassLoader()); FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext()); - FunctionUtils.openFunction(function, parameters); + FunctionUtils.openFunction(function, openContext); } @Override diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java index f6a039febcf39..574b2594b5549 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/match/PatternProcessFunctionRunner.java @@ -18,9 +18,9 @@ package org.apache.flink.table.runtime.operators.match; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.cep.functions.PatternProcessFunction; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.generated.GeneratedFunction; import org.apache.flink.util.Collector; @@ -44,10 +44,10 @@ public PatternProcessFunctionRunner( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.function = generatedFunction.newInstance(getRuntimeContext().getUserCodeClassLoader()); FunctionUtils.setFunctionRuntimeContext(function, getRuntimeContext()); - FunctionUtils.openFunction(function, parameters); + FunctionUtils.openFunction(function, openContext); } @Override diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java index 3528279d6f774..7f91ca2b7b327 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/AbstractRowTimeUnboundedPrecedingOver.java @@ -19,13 +19,13 @@ package org.apache.flink.table.runtime.operators.over; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; @@ -96,7 +96,7 @@ public AbstractRowTimeUnboundedPrecedingOver( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); function.open(new PerKeyStateDataViewStore(getRuntimeContext())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java index d93fae89460c0..39405ec8a05b4 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRangeBoundedPrecedingFunction.java @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.operators.over; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; @@ -25,7 +26,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.operators.TimestampedCollector; import org.apache.flink.table.data.RowData; @@ -84,7 +84,7 @@ public ProcTimeRangeBoundedPrecedingFunction( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); function.open(new PerKeyStateDataViewStore(getRuntimeContext())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java index 66bfc5cb863f7..0453d32806904 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeRowsBoundedPrecedingFunction.java @@ -18,6 +18,7 @@ package org.apache.flink.table.runtime.operators.over; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; @@ -25,7 +26,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.utils.JoinedRowData; @@ -89,7 +89,7 @@ public ProcTimeRowsBoundedPrecedingFunction( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); function.open(new PerKeyStateDataViewStore(getRuntimeContext())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java index c8e3de37ff858..c64651900eca7 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/ProcTimeUnboundedPrecedingFunction.java @@ -18,10 +18,10 @@ package org.apache.flink.table.runtime.operators.over; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.utils.JoinedRowData; @@ -61,7 +61,7 @@ public ProcTimeUnboundedPrecedingFunction( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); function.open(new PerKeyStateDataViewStore(getRuntimeContext())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java index add1a529f729b..bf5a3d5ca7fa1 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRangeBoundedPrecedingFunction.java @@ -19,13 +19,13 @@ package org.apache.flink.table.runtime.operators.over; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; @@ -110,7 +110,7 @@ public RowTimeRangeBoundedPrecedingFunction( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); function.open(new PerKeyStateDataViewStore(getRuntimeContext())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java index d491735177146..74ffb428c2dc1 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/over/RowTimeRowsBoundedPrecedingFunction.java @@ -19,13 +19,13 @@ package org.apache.flink.table.runtime.operators.over; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.table.data.RowData; @@ -114,7 +114,7 @@ public RowTimeRowsBoundedPrecedingFunction( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { function = genAggsHandler.newInstance(getRuntimeContext().getUserCodeClassLoader()); function.open(new PerKeyStateDataViewStore(getRuntimeContext())); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java index a097b0470da52..03975e1fbf286 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AbstractTopNFunction.java @@ -18,12 +18,12 @@ package org.apache.flink.table.runtime.operators.rank; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.configuration.Configuration; import org.apache.flink.metrics.Counter; import org.apache.flink.metrics.Gauge; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; @@ -138,8 +138,8 @@ public abstract class AbstractTopNFunction extends KeyedProcessFunction stateDesc = new ValueStateDescriptor<>("counterState", Types.INT); if (ttlConfig.isEnabled()) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java index 4186434dc2936..afbdd353d3444 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/AppendOnlyTopNFunction.java @@ -19,12 +19,12 @@ package org.apache.flink.table.runtime.operators.rank; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; @@ -93,8 +93,8 @@ public AppendOnlyTopNFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); int lruCacheSize = Math.max(1, (int) (cacheSize / getDefaultTopNSize())); CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); if (ttlConfig.isEnabled()) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/FastTop1Function.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/FastTop1Function.java index e7d1876a381a0..cffb136471530 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/FastTop1Function.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/FastTop1Function.java @@ -19,11 +19,11 @@ package org.apache.flink.table.runtime.operators.rank; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -91,8 +91,8 @@ public FastTop1Function( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); int lruCacheSize = Math.max(1, (int) (cacheSize / getDefaultTopNSize())); CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); if (ttlConfig.isEnabled()) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java index b9d6b6dfdca61..f354d73176f4c 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/RetractableTopNFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.operators.rank; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.StateTtlConfig; @@ -27,7 +28,6 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.typeutils.ListTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.RowDataUtil; import org.apache.flink.table.runtime.generated.GeneratedRecordEqualiser; @@ -110,8 +110,8 @@ public RetractableTopNFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); // compile equaliser equaliser = generatedEqualiser.newInstance(getRuntimeContext().getUserCodeClassLoader()); diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java index a047b727fbf0f..4ab664dda7891 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/rank/UpdatableTopNFunction.java @@ -19,6 +19,7 @@ package org.apache.flink.table.runtime.operators.rank; import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.StateTtlConfig; @@ -27,7 +28,6 @@ import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.typeutils.TupleTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -128,8 +128,8 @@ public UpdatableTopNFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); int lruCacheSize = Math.max(1, (int) (cacheSize / getDefaultTopNSize())); CacheBuilder cacheBuilder = CacheBuilder.newBuilder(); if (ttlConfig.isEnabled()) { diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java index 90559a5ce26e9..55b7ca4c7fef1 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperator.java @@ -18,8 +18,8 @@ package org.apache.flink.table.runtime.operators.wmassigners; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.AbstractStreamOperator; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; @@ -95,7 +95,7 @@ public void open() throws Exception { } FunctionUtils.setFunctionRuntimeContext(watermarkGenerator, getRuntimeContext()); - FunctionUtils.openFunction(watermarkGenerator, new Configuration()); + FunctionUtils.openFunction(watermarkGenerator, DefaultOpenContext.INSTANCE); } @Override diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java index 1e52e991efda8..f394a8a2588d2 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/AsyncLookupJoinHarnessTest.java @@ -19,9 +19,10 @@ package org.apache.flink.table.runtime.operators.join; import org.apache.flink.api.common.functions.AbstractRichFunction; +import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.functions.async.ResultFuture; @@ -307,11 +308,11 @@ public void testCloseAsyncLookupJoinRunner() throws Exception { closeAsyncLookupJoinRunner(joinRunner); joinRunner.setRuntimeContext(new MockStreamingRuntimeContext(false, 1, 0)); - joinRunner.open(new Configuration()); + joinRunner.open(DefaultOpenContext.INSTANCE); assertThat(joinRunner.getAllResultFutures()).isNotNull(); closeAsyncLookupJoinRunner(joinRunner); - joinRunner.open(new Configuration()); + joinRunner.open(DefaultOpenContext.INSTANCE); joinRunner.asyncInvoke(row(1, "a"), new TestingFetcherResultFuture()); assertThat(joinRunner.getAllResultFutures()).isNotNull(); closeAsyncLookupJoinRunner(joinRunner); @@ -365,8 +366,8 @@ public static final class TestingFetcherFunction extends AbstractRichFunction private transient ExecutorService executor; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); // generate unordered result for async lookup this.executor = Executors.newFixedThreadPool(2); } diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/KeyedLookupJoinHarnessTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/KeyedLookupJoinHarnessTest.java index dc807a61124b8..ade839d64263a 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/KeyedLookupJoinHarnessTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/KeyedLookupJoinHarnessTest.java @@ -18,10 +18,10 @@ package org.apache.flink.table.runtime.operators.join; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; @@ -454,7 +454,7 @@ public static class TestingEvolvingOutputFetcherFunction private transient Map accessCounter; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { baseData.clear(); baseData.put(1, Collections.singletonList(GenericRowData.of(1, fromString("Julian")))); baseData.put( diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java index bb8e9aa34594f..450840201b842 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/wmassigners/WatermarkAssignerOperatorTest.java @@ -18,8 +18,8 @@ package org.apache.flink.table.runtime.operators.wmassigners; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.watermarkstatus.WatermarkStatus; @@ -259,8 +259,8 @@ private MyWatermarkGenerator(int watermarkFieldIndex) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); if (closeCalled) { fail("Close called before open."); } diff --git a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java index 65751c7d7f16f..ca7a032d323f8 100644 --- a/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java +++ b/flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/test/streaming/runtime/util/TestListResultSink.java @@ -17,7 +17,7 @@ package org.apache.flink.test.streaming.runtime.util; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import java.util.ArrayList; @@ -39,8 +39,8 @@ public TestListResultSink() { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); } @Override diff --git a/flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/state/benchmark/RescalingBenchmarkTest.java b/flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/state/benchmark/RescalingBenchmarkTest.java index 4328193a76b37..2c46caabe1bb8 100644 --- a/flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/state/benchmark/RescalingBenchmarkTest.java +++ b/flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/state/benchmark/RescalingBenchmarkTest.java @@ -19,11 +19,11 @@ package org.apache.flink.state.benchmark; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; import org.apache.flink.runtime.state.storage.FileSystemCheckpointStorage; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; @@ -121,8 +121,8 @@ private static class TestKeyedFunction extends KeyedProcessFunction randomState; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); randomState = this.getRuntimeContext() .getState(new ValueStateDescriptor<>("RandomState", Integer.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/graph/TestEventSource.java b/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/graph/TestEventSource.java index 6a778770558a2..d8fa28ca7e82f 100644 --- a/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/graph/TestEventSource.java +++ b/flink-tests/src/test/java/org/apache/flink/runtime/operators/lifecycle/graph/TestEventSource.java @@ -17,7 +17,7 @@ package org.apache.flink.runtime.operators.lifecycle.graph; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.runtime.operators.lifecycle.command.TestCommand; import org.apache.flink.runtime.operators.lifecycle.command.TestCommandDispatcher; import org.apache.flink.runtime.operators.lifecycle.command.TestCommandDispatcher.CommandExecutor; @@ -60,8 +60,8 @@ public TestEventSource( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.isRunning = true; this.scheduledCommands = new LinkedBlockingQueue<>(); this.commandExecutor = cmd -> scheduledCommands.add(cmd); diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java index 15116df2248ad..9269981c095f3 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorErrorITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.DoubleCounter; import org.apache.flink.api.common.accumulators.LongCounter; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; @@ -111,7 +112,7 @@ private static class FaultyAccumulatorUsingMapper extends RichMapFunction distinctWords; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { // Add counters using convenience functions this.cntNumLines = getRuntimeContext().getIntCounter("num-lines"); @@ -193,7 +193,7 @@ private static class CountWords private IntCounter combineCalls; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { this.reduceCalls = getRuntimeContext().getIntCounter("reduce-calls"); this.combineCalls = getRuntimeContext().getIntCounter("combine-calls"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java index 018d11e25de6a..584a7bd185666 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorIterativeITCase.java @@ -19,11 +19,11 @@ package org.apache.flink.test.accumulators; import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; @@ -61,7 +61,7 @@ static final class SumReducer extends RichGroupReduceFunction private IntCounter testCounter = new IntCounter(); @Override - public void open(Configuration config) throws Exception { + public void open(OpenContext openContext) throws Exception { getRuntimeContext().addAccumulator(ACC_NAME, this.testCounter); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java index 1b0f2995c60c5..d048048f8a2e6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/accumulators/AccumulatorLiveITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.Plan; import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.io.OutputFormat; import org.apache.flink.api.common.time.Deadline; @@ -219,7 +220,7 @@ private static class NotifyingMapper extends RichFlatMapFunction> zs; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.zs = getRuntimeContext().getBroadcastVariable("z"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java index 04983882e0d2d..d9e378fe0d897 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastUnionITCase.java @@ -18,11 +18,11 @@ package org.apache.flink.test.broadcastvars; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.junit.Assert; @@ -55,7 +55,7 @@ private static class Mapper extends RichMapFunction { private List values; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { values = getRuntimeContext().getBroadcastVariable(BC_NAME); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java index fe5e54e4e169b..85ff136d01fbc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/broadcastvars/BroadcastVarInitializationITCase.java @@ -19,12 +19,12 @@ package org.apache.flink.test.broadcastvars; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; @@ -64,7 +64,7 @@ private static class PickOneAllReduce extends RichGroupReduceFunction private static final long serialVersionUID = 1L; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { synchronized (this) { wait(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java index 3d729ae24d415..70ab671873217 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/cancelling/MapCancelingITCase.java @@ -19,10 +19,10 @@ package org.apache.flink.test.cancelling; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.DiscardingOutputFormat; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.InfiniteIntegerInputFormat; import org.junit.Test; @@ -109,7 +109,7 @@ private static final class StuckInOpenIdentityMapper extends RichMapFunction private static final long serialVersionUID = 1L; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { synchronized (this) { wait(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ApproximateLocalRecoveryDownstreamITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ApproximateLocalRecoveryDownstreamITCase.java index 68651b78fa19c..64df7a8d92a17 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ApproximateLocalRecoveryDownstreamITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ApproximateLocalRecoveryDownstreamITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.configuration.Configuration; @@ -239,7 +240,7 @@ private static class FailingMapper extends RichMapFunction { } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { failer = getRuntimeContext().getIndexOfThisSubtask() == 0; } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogRecoveryITCaseBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogRecoveryITCaseBase.java index ae1bcfbe815a7..b5bc898954a9e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogRecoveryITCaseBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ChangelogRecoveryITCaseBase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobSubmissionResult; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; @@ -412,7 +413,7 @@ protected static class CountFunction private ValueState countState; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.countState = getRuntimeContext() .getState(new ValueStateDescriptor<>("countState", Integer.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CheckpointAfterAllTasksFinishedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CheckpointAfterAllTasksFinishedITCase.java index 3520f186074e1..893bde097c8ac 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CheckpointAfterAllTasksFinishedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CheckpointAfterAllTasksFinishedITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.Configuration; @@ -282,8 +283,8 @@ public CollectSink(SharedReference> result) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java index a8c1531307ccf..f025ba904e8c7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/CoStreamCheckpointingITCase.java @@ -18,12 +18,12 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -294,7 +294,7 @@ private static class OnceFailingReducer extends RichReduceFunction } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { long failurePosMin = (long) (0.4 * numElements / getRuntimeContext().getNumberOfParallelSubtasks()); long failurePosMax = diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java index c72b7c3dbbb21..19d704eca0e28 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ContinuousFileProcessingCheckpointITCase.java @@ -19,12 +19,12 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.io.FilePathFilter; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.java.io.TextInputFormat; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.ConfigConstants; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.execution.SuppressRestartsException; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.datastream.DataStream; @@ -191,7 +191,7 @@ private static class TestingSinkFunction extends RichSinkFunction } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { // this sink can only work with DOP 1 assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java index aa570070d15d5..274603bf9baba 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeAllWindowCheckpointingITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; @@ -106,7 +107,7 @@ public void testTumblingTimeWindow() { private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( 1, getRuntimeContext().getNumberOfParallelSubtasks()); open = true; @@ -182,7 +183,7 @@ public void testSlidingTimeWindow() { private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( 1, getRuntimeContext().getNumberOfParallelSubtasks()); open = true; @@ -264,7 +265,7 @@ public Tuple2 reduce( private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( 1, getRuntimeContext().getNumberOfParallelSubtasks()); open = true; @@ -344,7 +345,7 @@ public Tuple2 reduce( private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( 1, getRuntimeContext().getNumberOfParallelSubtasks()); open = true; diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java index 786e46233369f..763eedf6323c1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/EventTimeWindowCheckpointingITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ValueState; @@ -305,7 +306,7 @@ public void testTumblingTimeWindow() { private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); @@ -396,7 +397,7 @@ public void doTestTumblingTimeWindowWithKVState(int maxParallelism) { private ValueState count; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); @@ -483,7 +484,7 @@ public void testSlidingTimeWindow() { private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); @@ -569,7 +570,7 @@ public Tuple2 reduce( private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); @@ -655,7 +656,7 @@ public Tuple2 reduce( private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java index a68c4d8a67693..03734364df7a8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/KeyedStateCheckpointingITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.CheckpointListener; @@ -295,7 +296,7 @@ private static class OnceFailingPartitionedSum } @Override - public void open(Configuration parameters) throws IOException { + public void open(OpenContext openContext) throws IOException { sum = getRuntimeContext().getState(new ValueStateDescriptor<>("my_state", Long.class)); } @@ -342,7 +343,7 @@ private static class CounterSink extends RichSinkFunction> private transient ValueState bCounts; @Override - public void open(Configuration parameters) throws IOException { + public void open(OpenContext openContext) throws IOException { aCounts = getRuntimeContext() .getState(new ValueStateDescriptor<>("a", NonSerializableLong.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java index ee54c01eb8241..00f256c037f06 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ManualCheckpointITCase.java @@ -21,12 +21,12 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.connector.source.mocks.MockSource; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.state.CheckpointStorage; @@ -158,7 +158,7 @@ private static final class StatefulMapper extends RichFlatMapFunction count; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { count = getRuntimeContext() .getState( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ProcessingTimeWindowCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ProcessingTimeWindowCheckpointingITCase.java index 7e4fe0aa16b96..0c803b04c2e3f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ProcessingTimeWindowCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/ProcessingTimeWindowCheckpointingITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple; @@ -105,7 +106,7 @@ public void testTumblingProcessingTimeWindow() { private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); @@ -166,7 +167,7 @@ public void testSlidingProcessingTimeWindow() { private boolean open = false; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { assertEquals( PARALLELISM, getRuntimeContext().getNumberOfParallelSubtasks()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java index 0954533502741..b792e5fececbd 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/RegionFailoverITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -356,8 +357,8 @@ private static class FailingMapperFunction private ValueState valueState; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); valueState = getRuntimeContext() .getState(new ValueStateDescriptor<>("value", Integer.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java index 705ba21770d6f..e7f148a50701f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointFormatITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; @@ -349,7 +350,7 @@ private static final class StatefulCounter extends RichMapFunction { private ValueState counter; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { counter = getRuntimeContext() .getState( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java index 2a2b92522f49a..9570327cf931f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.JobStatus; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -481,7 +482,7 @@ public void testTriggerSavepointAndResumeWithNoClaim() throws Exception { private ListState last; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { // we use list state here to create sst files of a significant size // if sst files do not reach certain thresholds they are not stored // in files, but as a byte stream in checkpoints metadata @@ -1478,7 +1479,7 @@ private static class StatefulCounter extends RichMapFunction private byte[] data; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { if (data == null) { // We need this to be large, because we want to test with files Random rand = new Random(getRuntimeContext().getIndexOfThisSubtask()); @@ -1716,7 +1717,7 @@ private static class DuplicateFilter extends RichFlatMapFunction operatorState; @Override - public void open(Configuration configuration) { + public void open(OpenContext openContext) { operatorState = this.getRuntimeContext().getState(DESCRIPTOR); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java index 5f1135ca03375..732d78b59b5ff 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StateCheckpointedITCase.java @@ -18,12 +18,12 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ValueState; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -330,7 +330,7 @@ private static class OnceFailingAggregator extends RichFlatMapFunction("state-name", LongSerializer.INSTANCE); @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); getRuntimeContext() .addAccumulator(SUCCESSFUL_RESTORE_CHECK_ACCUMULATOR, new IntCounter()); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StatefulJobWBroadcastStateMigrationITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StatefulJobWBroadcastStateMigrationITCase.java index aad555cf1f5a4..3154b621e005a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StatefulJobWBroadcastStateMigrationITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StatefulJobWBroadcastStateMigrationITCase.java @@ -19,12 +19,12 @@ package org.apache.flink.test.checkpointing; import org.apache.flink.FlinkVersion; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend; import org.apache.flink.runtime.state.StateBackendLoader; import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; @@ -362,8 +362,8 @@ private static class CheckpointingKeyedBroadcastFunction private MapStateDescriptor secondStateDesc; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); firstStateDesc = new MapStateDescriptor<>( @@ -407,8 +407,8 @@ private static class CheckpointingKeyedSingleBroadcastFunction private MapStateDescriptor stateDesc; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); stateDesc = new MapStateDescriptor<>( @@ -456,8 +456,8 @@ private static class CheckingKeyedBroadcastFunction } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); firstStateDesc = new MapStateDescriptor<>( @@ -521,8 +521,8 @@ private static class CheckingKeyedSingleBroadcastFunction } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); stateDesc = new MapStateDescriptor<>( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java index a000925684ac7..2f332c259916f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointNotifierITCase.java @@ -18,13 +18,13 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -196,7 +196,7 @@ private static class GeneratingSourceFunction extends RichSourceFunction } @Override - public void open(Configuration parameters) throws IOException { + public void open(OpenContext openContext) throws IOException { step = getRuntimeContext().getNumberOfParallelSubtasks(); // if index has been restored, it is not 0 any more diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java index db2ed4c68f84d..995899dfa95e7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/StreamCheckpointingITCase.java @@ -18,11 +18,11 @@ package org.apache.flink.test.checkpointing; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -145,7 +145,7 @@ public void close() throws IOException { } @Override - public void open(Configuration parameters) throws IOException { + public void open(OpenContext openContext) throws IOException { step = getRuntimeContext().getNumberOfParallelSubtasks(); if (index == 0) { index = getRuntimeContext().getIndexOfThisSubtask(); @@ -256,7 +256,7 @@ private static class OnceFailingPrefixCounter extends RichMapFunction state; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); state = getRuntimeContext() .getState( diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointRescaleITCase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointRescaleITCase.java index 1f5d4c73afc5f..40527e6b3249b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointRescaleITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointRescaleITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -32,7 +33,6 @@ import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.api.common.typeutils.base.LongSerializer; import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -642,7 +642,7 @@ private static class StatefulKeyedMap extends RichMapFunction { ValueState state; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { state = getRuntimeContext().getState(DESC); } @@ -667,8 +667,8 @@ private static class InputCountFunction extends RichMapFunction private ListState state; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); getRuntimeContext().addAccumulator(NUM_INPUTS, numInputCounter); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointTestBase.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointTestBase.java index 7f1b234e783ea..5e5f1bb90186e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointTestBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/UnalignedCheckpointTestBase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.accumulators.IntCounter; import org.apache.flink.api.common.accumulators.LongCounter; import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -979,8 +980,8 @@ protected VerifyingSinkBase(long minCheckpoints, long checkpointingInterval) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); getRuntimeContext().addAccumulator(NUM_OUTPUTS, numOutputCounter); getRuntimeContext().addAccumulator(NUM_OUT_OF_ORDER, outOfOrderCounter); getRuntimeContext().addAccumulator(NUM_DUPLICATES, duplicatesCounter); diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/AccumulatingIntegerSink.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/AccumulatingIntegerSink.java index df7caa02fa28e..4053897a9f822 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/AccumulatingIntegerSink.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/AccumulatingIntegerSink.java @@ -18,8 +18,8 @@ package org.apache.flink.test.checkpointing.utils; import org.apache.flink.api.common.accumulators.ListAccumulator; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -48,8 +48,8 @@ public AccumulatingIntegerSink(int delayMillis) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); getRuntimeContext().addAccumulator(ACCUMULATOR_NAME, accumulator); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/CancellingIntegerSource.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/CancellingIntegerSource.java index f3197f83d4e67..2ded0cd772945 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/CancellingIntegerSource.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/CancellingIntegerSource.java @@ -17,10 +17,10 @@ package org.apache.flink.test.checkpointing.utils; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -96,8 +96,8 @@ public void initializeState(FunctionInitializationContext context) throws Except } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); } @Override diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java index 624e96355c3f0..e18817766d2c5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/FailingSource.java @@ -18,10 +18,10 @@ package org.apache.flink.test.checkpointing.utils; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; @@ -77,7 +77,7 @@ public FailingSource( } @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { // non-parallel source assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/MigrationTestUtils.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/MigrationTestUtils.java index 9fd021eb65a46..8317cb94e3ee1 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/MigrationTestUtils.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/MigrationTestUtils.java @@ -19,11 +19,11 @@ package org.apache.flink.test.checkpointing.utils; import org.apache.flink.api.common.accumulators.IntCounter; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -324,8 +324,8 @@ public static class AccumulatorCountingSink extends RichSinkFunction { int count = 0; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); getRuntimeContext().addAccumulator(NUM_ELEMENTS_ACCUMULATOR, new IntCounter()); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java index 978a3498c74bd..da0c0e6dab028 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java +++ b/flink-tests/src/test/java/org/apache/flink/test/checkpointing/utils/ValidatingSink.java @@ -18,7 +18,7 @@ package org.apache.flink.test.checkpointing.utils; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.checkpoint.ListCheckpointed; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import org.apache.flink.test.util.SuccessException; @@ -77,7 +77,7 @@ public ValidatingSink( } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { // this sink can only work with DOP 1 assertEquals(1, getRuntimeContext().getNumberOfParallelSubtasks()); if (usingProcessingTime && resultChecker.checkResult(windowCounts)) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java index 360c1317c73a3..5f4bb72701292 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CheckpointingCustomKvStateProgram.java @@ -19,6 +19,7 @@ package org.apache.flink.test.classloading.jar; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -31,7 +32,6 @@ import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -129,7 +129,7 @@ private static class ReducingStateFlatMap private boolean restored = false; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { ReducingStateDescriptor stateDescriptor = new ReducingStateDescriptor<>( "reducing-state", new ReduceSum(), CustomIntSerializer.INSTANCE); diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java index 1c0845c5f4291..990677ef6a005 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/CustomKvStateProgram.java @@ -19,13 +19,13 @@ package org.apache.flink.test.classloading.jar; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.state.ReducingState; import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.filesystem.FsStateBackend; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -90,7 +90,7 @@ private static class ReducingStateFlatMap private transient ReducingState kvState; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { ReducingStateDescriptor stateDescriptor = new ReducingStateDescriptor<>("reducing-state", new ReduceSum(), Integer.class); diff --git a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java index c3ffffbc11d24..d601421de7040 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/classloading/jar/KMeansForTest.java @@ -20,13 +20,13 @@ import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.api.common.accumulators.SimpleAccumulator; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import java.util.Collection; @@ -196,7 +196,7 @@ public static final class SelectNearestCenter /** Reads the centroid values from a broadcast variable into a collection. */ @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); this.acc = new CustomAccumulator(); getRuntimeContext().addAccumulator("myAcc", this.acc); diff --git a/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java b/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java index 842d25a8a9efe..f062d241b24f9 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/distributedcache/DistributedCacheTest.java @@ -18,10 +18,10 @@ package org.apache.flink.test.distributedcache; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple1; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.AbstractTestBase; import org.apache.flink.util.Collector; @@ -68,7 +68,7 @@ private static class WordChecker extends RichFlatMapFunction wordList = new ArrayList<>(); @Override - public void open(Configuration conf) throws IOException { + public void open(OpenContext openContext) throws IOException { File file = getRuntimeContext().getDistributedCache().getFile("cache_test"); try (BufferedReader reader = new BufferedReader(new FileReader(file))) { String tempString; diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java index 16edf83968846..2af42e42efccc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/BulkIterationWithAllReducerITCase.java @@ -18,12 +18,12 @@ package org.apache.flink.test.iterative; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.io.LocalCollectionOutputFormat; import org.apache.flink.api.java.operators.IterativeDataSet; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.JavaProgramTestBase; import org.apache.flink.util.Collector; @@ -61,7 +61,7 @@ private static class PickOneAllReduce extends RichGroupReduceFunction bc = getRuntimeContext().getBroadcastVariable("bc"); this.bcValue = bc.isEmpty() ? null : bc.get(0); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java index bb27b3f388376..deb5111d8020e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/DanglingPageRankITCase.java @@ -23,12 +23,12 @@ import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichCoGroupFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.test.util.MultipleProgramsTestBase; @@ -158,8 +158,7 @@ public void join( private double danglingRankFactor; @Override - public void open(Configuration parameters) - throws Exception { + public void open(OpenContext openContext) throws Exception { int currentIteration = getIterationRuntimeContext() .getSuperstepNumber(); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java index d265a8a6c5c7c..636b3ba7d10dc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorConvergenceITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.aggregators.LongSumAggregator; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichJoinFunction; import org.apache.flink.api.java.DataSet; @@ -27,7 +28,6 @@ import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.LongValue; @@ -283,7 +283,7 @@ public MinimumIdFilter(String aggName) { } @Override - public void open(Configuration conf) { + public void open(OpenContext openContext) { aggr = getIterationRuntimeContext().getIterationAggregator(aggName); } @@ -315,7 +315,7 @@ public MinimumIdFilterCounting(String aggName) { } @Override - public void open(Configuration conf) { + public void open(OpenContext openContext) { final int superstep = getIterationRuntimeContext().getSuperstepNumber(); aggr = getIterationRuntimeContext().getIterationAggregator(aggName); diff --git a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java index 6c4867e5c8aa2..bd76b5ec2b5cc 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/iterative/aggregators/AggregatorsITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.aggregators.ConvergenceCriterion; import org.apache.flink.api.common.aggregators.LongSumAggregator; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.common.functions.RichMapFunction; @@ -29,7 +30,6 @@ import org.apache.flink.api.java.operators.DeltaIteration; import org.apache.flink.api.java.operators.IterativeDataSet; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.util.MultipleProgramsTestBase; import org.apache.flink.types.LongValue; @@ -93,7 +93,7 @@ public void testDistributedCacheWithIterations() throws Exception { .filter( new RichFilterFunction() { @Override - public void open(Configuration parameters) + public void open(OpenContext openContext) throws Exception { File file = getRuntimeContext() @@ -356,7 +356,7 @@ private static final class SubtractOneMap extends RichMapFunction ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java index 556d2fb1cf891..a69a1c9cf6a63 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/CrossITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.operators; import org.apache.flink.api.common.functions.CrossFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichCrossFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -26,7 +27,6 @@ import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.java.tuple.Tuple6; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.MultipleProgramsTestBase; @@ -485,7 +485,7 @@ private static class Tuple5CrossBC private int broadcast = 42; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java index c49c3f06e2551..4051dc0df6f97 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/FilterITCase.java @@ -19,11 +19,11 @@ package org.apache.flink.test.operators; import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFilterFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.MultipleProgramsTestBase; @@ -264,7 +264,7 @@ private static class RichFilter1 extends RichFilterFunction ints = this.getRuntimeContext().getBroadcastVariable("ints"); for (int i : ints) { literal = literal < i ? i : literal; @@ -307,7 +307,7 @@ private static class RichFilter2 extends RichFilterFunction ints = this.getRuntimeContext().getBroadcastVariable("ints"); for (Integer i : ints) { broadcastSum += i; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java index f0c1c17879bf0..60c79136c400e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/FlatMapITCase.java @@ -19,11 +19,11 @@ package org.apache.flink.test.operators; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; import org.apache.flink.test.util.MultipleProgramsTestBase; @@ -396,7 +396,7 @@ private static class RichFlatMapper1 private Integer f2Replace = 0; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for (Integer i : ints) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java index c02f20ebf6069..9f5636d1ec49a 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/GroupReduceITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.operators.Order; @@ -1507,7 +1508,7 @@ private static class BCTuple3GroupReduce private String f2Replace = ""; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java index f2026e1e65ef6..03ab690c6f46b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/JoinITCase.java @@ -22,6 +22,7 @@ import org.apache.flink.api.common.distributions.DataDistribution; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -34,7 +35,6 @@ import org.apache.flink.api.java.tuple.Tuple6; import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.java.utils.DataSetUtils; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.test.operators.util.CollectionDataSets; @@ -937,7 +937,7 @@ private static class T3T5BCJoin private int broadcast; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java index 52037e12fb639..2577888e86db6 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/MapITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.operators; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; @@ -483,7 +484,7 @@ private static class RichMapper1 private Integer f2Replace = 0; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for (Integer i : ints) { @@ -527,12 +528,6 @@ private static class RichMapper2 extends RichMapFunction, Tuple3> { private static final long serialVersionUID = 1L; - @Override - public void open(Configuration config) { - int val = config.getInteger(TEST_KEY, -1); - Assert.assertEquals(TEST_VALUE, val); - } - @Override public Tuple3 map(Tuple3 value) { return value; diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java index edfd57a58c0cd..c80001721a3a7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/OuterJoinITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.InvalidProgramException; import org.apache.flink.api.common.functions.FlatJoinFunction; import org.apache.flink.api.common.functions.JoinFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichFlatJoinFunction; import org.apache.flink.api.common.operators.base.JoinOperatorBase.JoinHint; import org.apache.flink.api.java.DataSet; @@ -31,7 +32,6 @@ import org.apache.flink.api.java.tuple.Tuple5; import org.apache.flink.api.java.tuple.Tuple7; import org.apache.flink.api.java.typeutils.GenericTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; import org.apache.flink.test.operators.util.CollectionDataSets.POJO; @@ -753,7 +753,7 @@ private static class T3T5BCJoin private int broadcast; @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { Collection ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; for (Integer i : ints) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java index c432c09dfc198..e4eaf4827bb65 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/operators/ReduceITCase.java @@ -20,6 +20,7 @@ import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.functions.RichReduceFunction; import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint; @@ -29,7 +30,6 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; import org.apache.flink.api.java.tuple.Tuple5; -import org.apache.flink.configuration.Configuration; import org.apache.flink.test.operators.util.CollectionDataSets; import org.apache.flink.test.operators.util.CollectionDataSets.CustomType; import org.apache.flink.test.operators.util.CollectionDataSets.PojoWithDateAndEnum; @@ -504,7 +504,7 @@ private static class BCTuple3Reduce extends RichReduceFunction ints = this.getRuntimeContext().getBroadcastVariable("ints"); int sum = 0; diff --git a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java index 2143c1af1dd51..4c9922d7c4a64 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/optimizer/examples/KMeansSingleStepTest.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.functions.GroupCombineFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.operators.GenericDataSourceBase; import org.apache.flink.api.common.operators.util.FieldList; @@ -30,7 +31,6 @@ import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.api.java.tuple.Tuple3; -import org.apache.flink.configuration.Configuration; import org.apache.flink.optimizer.plan.OptimizedPlan; import org.apache.flink.optimizer.plan.SingleInputPlanNode; import org.apache.flink.optimizer.plan.SinkPlanNode; @@ -242,7 +242,7 @@ private static final class SelectNearestCenter private Collection centroids; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.centroids = getRuntimeContext().getBroadcastVariable("centroids"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java index 854a4e0f2bee4..4f78ae4977b1f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/FastFailuresITCase.java @@ -19,9 +19,9 @@ package org.apache.flink.test.recovery; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.SinkFunction; @@ -54,7 +54,7 @@ public void testThis() throws Exception { new RichSourceFunction>() { @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { if (FAILURES_SO_FAR.incrementAndGet() <= NUM_FAILURES) { throw new RuntimeException("fail"); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java index 5aa2c1dde7383..b9483918e0291 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/recovery/TaskManagerProcessFailureStreamingRecoveryITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.recovery; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -198,7 +199,7 @@ public CheckpointedSink(long end) { } @Override - public void open(Configuration parameters) throws IOException { + public void open(OpenContext openContext) throws IOException { stepSize = getRuntimeContext().getNumberOfParallelSubtasks(); congruence = getRuntimeContext().getIndexOfThisSubtask(); toCollect = (end % stepSize > congruence) ? (end / stepSize + 1) : (end / stepSize); diff --git a/flink-tests/src/test/java/org/apache/flink/test/runtime/BatchShuffleITCaseBase.java b/flink-tests/src/test/java/org/apache/flink/test/runtime/BatchShuffleITCaseBase.java index 707b63ce7b310..0e6d24fd9e0d8 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/runtime/BatchShuffleITCaseBase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/runtime/BatchShuffleITCaseBase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.runtime; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.connector.source.Boundedness; @@ -156,7 +157,7 @@ private static class VerifySink extends RichSinkFunction { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { NUM_RECEIVED_RECORDS[getRuntimeContext().getIndexOfThisSubtask()] = 0; if (getRuntimeContext().getAttemptNumber() > 0 || getRuntimeContext().getIndexOfThisSubtask() != 0) { diff --git a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java index 846cad8d47160..89e789a6bb5f5 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/scheduling/ReactiveModeITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.test.scheduling; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.client.program.rest.RestClusterClient; import org.apache.flink.configuration.Configuration; @@ -249,7 +250,7 @@ private static class FailOnParallelExecutionSource extends RichParallelSourceFun private volatile boolean running = true; @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { if (getRuntimeContext().getNumberOfParallelSubtasks() > 1) { throw new IllegalStateException( "This is not supposed to be executed in parallel, despite extending the right base class."); diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java index dd90507ab94c4..679546de90d4b 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/TaskManagerWideRocksDbMemorySharingITCase.java @@ -18,6 +18,7 @@ package org.apache.flink.test.state; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -135,8 +136,8 @@ private JobGraph dag(RocksDBMemoryFactory memoryFactory) { private int payloadSize; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); this.state = getRuntimeContext() .getListState( diff --git a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java index a9b884a7f3288..6d6bd13590f4d 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java +++ b/flink-tests/src/test/java/org/apache/flink/test/state/operator/restore/keyed/KeyedJob.java @@ -18,6 +18,7 @@ package org.apache.flink.test.state.operator.restore.keyed; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ListState; @@ -173,7 +174,7 @@ private StatefulWindowFunction(ExecutionMode mode) { } @Override - public void open(Configuration config) { + public void open(OpenContext openContext) { this.state = getRuntimeContext() .getListState(new ListStateDescriptor<>("values", Integer.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java index 51b649f282d3e..f3e1939e98e20 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/StreamingOperatorsITCase.java @@ -19,8 +19,8 @@ package org.apache.flink.test.streaming.api; import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -70,8 +70,8 @@ public void testAsyncWaitOperator() throws Exception { transient ExecutorService executorService; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); executorService = Executors.newFixedThreadPool(numElements); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java index f200765bda8db..7c790dca6224f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/GetOperatorUniqueIDTest.java @@ -20,7 +20,7 @@ import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.util.TestLogger; @@ -62,8 +62,8 @@ public VerifyOperatorIDMapFunction(String expectedOperatorUniqueID) { } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); assertEquals( expectedOperatorUniqueID, diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java index 446b9cf416c60..3d87de184cdb0 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/ReinterpretDataStreamAsKeyedStreamITCase.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.eventtime.WatermarkGenerator; import org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.CheckpointListener; @@ -31,7 +32,6 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.state.FunctionInitializationContext; import org.apache.flink.runtime.state.FunctionSnapshotContext; import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; @@ -167,8 +167,8 @@ private static class ToPartitionFileSink extends RichSinkFunction( @@ -288,8 +288,8 @@ private static class TestBroadcastProcessFunction private transient MapStateDescriptor descriptor; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); descriptor = new MapStateDescriptor<>( diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java index 15bfb46761bd6..b88afcabd4ab2 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/ChainedRuntimeContextITCase.java @@ -17,9 +17,9 @@ package org.apache.flink.test.streaming.runtime; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; @@ -55,7 +55,7 @@ public void run(SourceContext ctx) throws Exception {} public void cancel() {} @Override - public void open(Configuration c) { + public void open(OpenContext openContext) { srcContext = getRuntimeContext(); } } @@ -68,7 +68,7 @@ public Integer map(Integer value) throws Exception { } @Override - public void open(Configuration c) { + public void open(OpenContext openContext) { mapContext = getRuntimeContext(); } } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/GlobalAggregateITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/GlobalAggregateITCase.java index 9e46507fbc1ea..5cf3b4a28d382 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/GlobalAggregateITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/GlobalAggregateITCase.java @@ -19,7 +19,7 @@ package org.apache.flink.test.streaming.runtime; import org.apache.flink.api.common.functions.AggregateFunction; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.runtime.taskexecutor.GlobalAggregateManager; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; @@ -79,8 +79,8 @@ public TestSourceFunction( } @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) getRuntimeContext(); aggregateManager = runtimeContext.getGlobalAggregateManager(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java index 2d98d601f5380..8690626b22566 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/PartitionerITCase.java @@ -18,12 +18,12 @@ package org.apache.flink.test.streaming.runtime; import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.java.tuple.Tuple1; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.streaming.runtime.util.NoOpIntMap; @@ -238,8 +238,8 @@ private static class SubtaskIndexAssigner private int indexOfSubtask; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); RuntimeContext runtimeContext = getRuntimeContext(); indexOfSubtask = runtimeContext.getIndexOfThisSubtask(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SortingBoundedInputITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SortingBoundedInputITCase.java index e7ad3eb69b08a..31007b3754ac7 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SortingBoundedInputITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/SortingBoundedInputITCase.java @@ -23,6 +23,7 @@ import org.apache.flink.api.common.eventtime.WatermarkGenerator; import org.apache.flink.api.common.eventtime.WatermarkOutput; import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; import org.apache.flink.api.common.state.ValueState; @@ -252,7 +253,7 @@ public void testBatchExecutionWithTimersOneInput() { private ValueState previousTimestampState; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { countState = getRuntimeContext() .getMapState( @@ -403,7 +404,7 @@ public void testBatchExecutionWithTimersTwoInput() { private ValueState previousTimestampState; @Override - public void open(Configuration parameters) { + public void open(OpenContext openContext) { countState = getRuntimeContext() .getMapState( diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java index 9d9697bf97d81..dcdc88849929f 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StateBackendITCase.java @@ -19,12 +19,12 @@ package org.apache.flink.test.streaming.runtime; import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.client.JobExecutionException; @@ -75,8 +75,8 @@ public void testStateBackendWithoutCheckpointing() throws Exception { private static final long serialVersionUID = 1L; @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); + public void open(OpenContext openContext) throws Exception { + super.open(openContext); getRuntimeContext() .getState( new ValueStateDescriptor<>("Test", Integer.class)); diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java index 2ab8cfe6606f0..65f375cd3bb30 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamTaskSelectiveReadingITCase.java @@ -18,9 +18,9 @@ package org.apache.flink.test.streaming.runtime; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; @@ -113,7 +113,7 @@ public TestSource(String name, T[] elements) { } @Override - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { this.context = getRuntimeContext(); } diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java index 483e730027dba..540d31184b10e 100644 --- a/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java +++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/util/ReceiveCheckNoOpSink.java @@ -18,7 +18,7 @@ package org.apache.flink.test.streaming.runtime.util; -import org.apache.flink.configuration.Configuration; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.streaming.api.functions.sink.RichSinkFunction; import java.util.ArrayList; @@ -38,7 +38,7 @@ public void invoke(T tuple) { received.add(tuple); } - public void open(Configuration conf) { + public void open(OpenContext openContext) { received = new ArrayList(); } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala index b4d2e48439503..973c9f88c5f91 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobSavepointMigrationITCase.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.scala.migration import org.apache.flink.FlinkVersion import org.apache.flink.api.common.accumulators.IntCounter -import org.apache.flink.api.common.functions.RichFlatMapFunction +import org.apache.flink.api.common.functions.{OpenContext, RichFlatMapFunction} import org.apache.flink.api.common.state.{ListState, ListStateDescriptor, ValueState, ValueStateDescriptor} import org.apache.flink.api.java.functions.KeySelector import org.apache.flink.api.java.tuple.Tuple2 @@ -289,8 +289,8 @@ class StatefulJobSavepointMigrationITCase(snapshotSpec: SnapshotSpec) private var count: Int = 0 @throws[Exception] - override def open(parameters: Configuration) { - super.open(parameters) + override def open(openContext: OpenContext) { + super.open(openContext) getRuntimeContext.addAccumulator( AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, new IntCounter) @@ -317,7 +317,7 @@ class StatefulJobSavepointMigrationITCase(snapshotSpec: SnapshotSpec) private var enumOneState: ValueState[CustomEnum] = _ private var enumThreeState: ValueState[CustomEnum] = _ - override def open(parameters: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { caseClassState = getRuntimeContext.getState( new ValueStateDescriptor[CustomCaseClass]( "caseClassState", diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobWBroadcastStateMigrationITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobWBroadcastStateMigrationITCase.scala index f65c18167c212..3fcb744fc9f84 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobWBroadcastStateMigrationITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/migration/StatefulJobWBroadcastStateMigrationITCase.scala @@ -19,7 +19,7 @@ package org.apache.flink.api.scala.migration import org.apache.flink.FlinkVersion import org.apache.flink.api.common.accumulators.IntCounter -import org.apache.flink.api.common.functions.RichFlatMapFunction +import org.apache.flink.api.common.functions.{OpenContext, RichFlatMapFunction} import org.apache.flink.api.common.state._ import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, TypeInformation} import org.apache.flink.api.java.functions.KeySelector @@ -364,8 +364,8 @@ private class AccumulatorCountingSink[T] extends RichSinkFunction[T] { private var count: Int = 0 @throws[Exception] - override def open(parameters: Configuration) { - super.open(parameters) + override def open(openContext: OpenContext) { + super.open(openContext) getRuntimeContext.addAccumulator( AccumulatorCountingSink.NUM_ELEMENTS_ACCUMULATOR, new IntCounter) @@ -392,7 +392,7 @@ class StatefulFlatMapper extends RichFlatMapFunction[(Long, Long), (Long, Long)] private var enumOneState: ValueState[CustomEnum] = _ private var enumThreeState: ValueState[CustomEnum] = _ - override def open(parameters: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { caseClassState = getRuntimeContext.getState( new ValueStateDescriptor[CustomCaseClass]( "caseClassState", diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala index 6e71a291ea81e..8f64bf58bfdf3 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CoGroupITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichCoGroupFunction +import org.apache.flink.api.common.functions.{OpenContext, RichCoGroupFunction} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.util.CollectionDataSets.CustomType @@ -163,7 +163,7 @@ class CoGroupITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mo (Int, Int, Int)] { private var broadcast = 41 - override def open(config: Configuration) { + override def open(openContext: OpenContext) { val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala broadcast = ints.sum } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala index 80cc4c94f54bf..432c3f50e9817 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/CrossITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichCrossFunction +import org.apache.flink.api.common.functions.{OpenContext, RichCrossFunction} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.util.CollectionDataSets.CustomType @@ -121,7 +121,7 @@ class CrossITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode (Int, Int, Int)] { private var broadcast = 41 - override def open(config: Configuration) { + override def open(openContext: OpenContext) { val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala broadcast = ints.sum } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala index 65fee0c64361f..856815a063539 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FilterITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichFilterFunction +import org.apache.flink.api.common.functions.{OpenContext, RichFilterFunction} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.configuration.Configuration @@ -147,7 +147,7 @@ class FilterITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mod val filterDs = ds .filter(new RichFilterFunction[(Int, Long, String)] { var literal = -1 - override def open(config: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { val ints = getRuntimeContext.getBroadcastVariable[Int]("ints") for (i <- ints.asScala) { literal = if (literal < i) i else literal diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala index abce9b26e24ea..f8f7da6149466 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/FlatMapITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichFlatMapFunction +import org.apache.flink.api.common.functions.{OpenContext, RichFlatMapFunction} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3 @@ -180,7 +180,7 @@ class FlatMapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mo MutableTuple3[Int, Long, String]] { private var f2Replace = 0 private val outTuple = MutableTuple3(0, 0L, "") - override def open(config: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { val ints = getRuntimeContext.getBroadcastVariable[Int]("ints").asScala f2Replace = ints.sum } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala index 5256c528d69c3..dee4751e9ee1e 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/GroupReduceITCase.scala @@ -216,7 +216,7 @@ class GroupReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBas .reduceGroup(new RichGroupReduceFunction[(Int, Long, String), (Int, Long, String)] { private var f2Replace = "" - override def open(config: Configuration) { + override def open(openContext: OpenContext) { val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala f2Replace = ints.sum + "" } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala index 5fb6ea438f65c..d6688c607eb2f 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/JoinITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichJoinFunction +import org.apache.flink.api.common.functions.{OpenContext, RichJoinFunction} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.configuration.Configuration @@ -171,7 +171,7 @@ class JoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) (String, String, Int)] { private var broadcast = 41 - override def open(config: Configuration) { + override def open(openContext: OpenContext) { val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala broadcast = ints.sum } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala index 26defae6289bd..a989130f0829a 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/MapITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichMapFunction +import org.apache.flink.api.common.functions.{OpenContext, RichMapFunction} import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.util.CollectionDataSets.MutableTuple3 @@ -184,7 +184,7 @@ class MapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) val bcMapDs = ds .map(new RichMapFunction[(Int, Long, String), (Int, Long, String)] { var f2Replace = 0 - override def open(config: Configuration): Unit = { + override def open(openContext: OpenContext): Unit = { val ints = getRuntimeContext.getBroadcastVariable[Int]("ints").asScala f2Replace = ints.sum } @@ -202,31 +202,4 @@ class MapITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mode) "Comment#9\n" + "55,6,Comment#10\n" + "55,6,Comment#11\n" + "55,6,Comment#12\n" + "55," + "6,Comment#13\n" + "55,6,Comment#14\n" + "55,6,Comment#15\n" } - - @Test - def testPassingConfigurationObject(): Unit = { - /* - * Test passing configuration object. - */ - val env = ExecutionEnvironment.getExecutionEnvironment - val ds = CollectionDataSets.getSmall3TupleDataSet(env) - val conf = new Configuration - val testKey = "testVariable" - val testValue = 666 - conf.setInteger(testKey, testValue) - val bcMapDs = ds - .map(new RichMapFunction[(Int, Long, String), (Int, Long, String)] { - override def open(config: Configuration): Unit = { - val fromConfig = config.getInteger(testKey, -1) - Assert.assertEquals(testValue, fromConfig) - } - override def map(in: (Int, Long, String)): (Int, Long, String) = { - in - } - }) - .withParameters(conf) - bcMapDs.writeAsCsv(resultPath, writeMode = WriteMode.OVERWRITE) - env.execute() - expected = "1,1,Hi\n" + "2,2,Hello\n" + "3,2,Hello world" - } } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala index ef7e458d6ad23..3e9d1e252bb8c 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/OuterJoinITCase.scala @@ -17,9 +17,9 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichJoinFunction +import org.apache.flink.api.common.functions.{OpenContext, RichJoinFunction} import org.apache.flink.api.common.io.OutputFormat -import org.apache.flink.api.scala.{ExecutionEnvironment, _} +import org.apache.flink.api.scala._ import org.apache.flink.api.scala.operators.ScalaCsvOutputFormat.{DEFAULT_FIELD_DELIMITER, DEFAULT_LINE_DELIMITER} import org.apache.flink.api.scala.util.CollectionDataSets import org.apache.flink.api.scala.util.CollectionDataSets.CustomType @@ -140,7 +140,7 @@ class OuterJoinITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase( (String, String, Int)] { private var broadcast = 41 - override def open(config: Configuration) { + override def open(openContext: OpenContext) { val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala broadcast = ints.sum } diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala index 15070d639b297..f9b2a4a3ed910 100644 --- a/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala +++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/operators/ReduceITCase.scala @@ -17,7 +17,7 @@ */ package org.apache.flink.api.scala.operators -import org.apache.flink.api.common.functions.RichReduceFunction +import org.apache.flink.api.common.functions.{OpenContext, RichReduceFunction} import org.apache.flink.api.common.operators.base.ReduceOperatorBase.CombineHint import org.apache.flink.api.scala._ import org.apache.flink.api.scala.util.CollectionDataSets @@ -168,7 +168,7 @@ class ReduceITCase(mode: TestExecutionMode) extends MultipleProgramsTestBase(mod .reduce(new RichReduceFunction[(Int, Long, String)] { private var f2Replace = "" - override def open(config: Configuration) { + override def open(openContext: OpenContext) { val ints = this.getRuntimeContext.getBroadcastVariable[Int]("ints").asScala f2Replace = ints.sum + "" } diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestArchiveJob.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestArchiveJob.java index 3d198b353f7a0..1db17ba2f9bcb 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestArchiveJob.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestArchiveJob.java @@ -18,6 +18,7 @@ package org.apache.flink.yarn.testjob; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; import org.apache.flink.configuration.Configuration; @@ -129,7 +130,7 @@ private static class SourceFunctionWithArchive extends RichSourceFunction this.returnType = returnType; } - public void open(Configuration parameters) throws Exception { + public void open(OpenContext openContext) throws Exception { for (Map.Entry entry : srcFiles.entrySet()) { Path path = Paths.get(resourcePath + File.separator + entry.getKey()); String content = new String(Files.readAllBytes(path)); diff --git a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestCacheJob.java b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestCacheJob.java index c6c109e66471d..1c9451bfb0a00 100644 --- a/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestCacheJob.java +++ b/flink-yarn-tests/src/test/java/org/apache/flink/yarn/testjob/YarnTestCacheJob.java @@ -18,10 +18,10 @@ package org.apache.flink.yarn.testjob; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.jobgraph.JobGraph; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink; @@ -69,7 +69,7 @@ private static class MapperFunction extends RichMapFunction { private static final long serialVersionUID = -1238033916372648233L; @Override - public void open(Configuration config) throws IOException { + public void open(OpenContext openContext) throws IOException { // access cached file via RuntimeContext and DistributedCache final File cacheFile = getRuntimeContext().getDistributedCache().getFile("cacheFile"); final FileInputStream inputStream = new FileInputStream(cacheFile);