diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/18509c9e-3250-4c52-91b9-11ccefc85db1 b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/18509c9e-3250-4c52-91b9-11ccefc85db1
index 793b3ba9b4c38..6760b22a4a3e6 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/18509c9e-3250-4c52-91b9-11ccefc85db1
+++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/18509c9e-3250-4c52-91b9-11ccefc85db1
@@ -51,25 +51,6 @@ org.apache.flink.api.connector.source.lib.NumberSequenceSource.getSplitSerialize
org.apache.flink.api.connector.source.lib.NumberSequenceSource.restoreEnumerator(org.apache.flink.api.connector.source.SplitEnumeratorContext, java.util.Collection): Argument leaf type org.apache.flink.api.connector.source.lib.NumberSequenceSource$NumberSequenceSplit does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.connector.source.lib.NumberSequenceSource.restoreEnumerator(org.apache.flink.api.connector.source.SplitEnumeratorContext, java.util.Collection): Returned leaf type org.apache.flink.api.connector.source.lib.NumberSequenceSource$NumberSequenceSplit does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.connector.source.lib.util.IteratorSourceReaderBase.pollNext(org.apache.flink.api.connector.source.ReaderOutput): Returned leaf type org.apache.flink.core.io.InputStatus does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.coGroup(org.apache.flink.api.java.DataSet): Returned leaf type org.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSets does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.write(org.apache.flink.api.common.io.FileOutputFormat, java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode): Argument leaf type org.apache.flink.core.fs.FileSystem$WriteMode does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.writeAsCsv(java.lang.String, java.lang.String, java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode): Argument leaf type org.apache.flink.core.fs.FileSystem$WriteMode does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.writeAsCsv(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode): Argument leaf type org.apache.flink.core.fs.FileSystem$WriteMode does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.writeAsFormattedText(java.lang.String, org.apache.flink.api.java.io.TextOutputFormat$TextFormatter): Argument leaf type org.apache.flink.api.java.io.TextOutputFormat$TextFormatter does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.writeAsFormattedText(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode, org.apache.flink.api.java.io.TextOutputFormat$TextFormatter): Argument leaf type org.apache.flink.api.java.io.TextOutputFormat$TextFormatter does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.writeAsFormattedText(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode, org.apache.flink.api.java.io.TextOutputFormat$TextFormatter): Argument leaf type org.apache.flink.core.fs.FileSystem$WriteMode does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.DataSet.writeAsText(java.lang.String, org.apache.flink.core.fs.FileSystem$WriteMode): Argument leaf type org.apache.flink.core.fs.FileSystem$WriteMode does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.aggregation.Aggregations.getFactory(): Returned leaf type org.apache.flink.api.java.aggregation.AggregationFunctionFactory does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.CoGroupOperator.getSemanticProperties(): Returned leaf type org.apache.flink.api.common.operators.DualInputSemanticProperties does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.CoGroupOperator.getSemanticProperties(): Returned leaf type org.apache.flink.api.common.operators.SemanticProperties does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets.where([I): Returned leaf type org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets.where([Ljava.lang.String;): Returned leaf type org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.JoinOperator$JoinOperatorSets.where(org.apache.flink.api.java.functions.KeySelector): Returned leaf type org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.Operator.getMinResources(): Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.Operator.getPreferredResources(): Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.where([I): Returned leaf type org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.where([Ljava.lang.String;): Returned leaf type org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.join.JoinOperatorSetsBase.where(org.apache.flink.api.java.functions.KeySelector): Returned leaf type org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.PojoTypeInfo.createPojoSerializer(org.apache.flink.api.common.serialization.SerializerConfig): Argument leaf type org.apache.flink.api.common.serialization.SerializerConfig does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.PojoTypeInfo.createPojoSerializer(org.apache.flink.api.common.serialization.SerializerConfig): Returned leaf type org.apache.flink.api.java.typeutils.runtime.PojoSerializer does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.ValueTypeInfo.createSerializer(org.apache.flink.api.common.serialization.SerializerConfig): Argument leaf type org.apache.flink.api.common.serialization.SerializerConfig does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
@@ -88,6 +69,10 @@ org.apache.flink.core.fs.FileSystem.initOutPathLocalFS(org.apache.flink.core.fs.
org.apache.flink.core.fs.FileSystem.initialize(org.apache.flink.configuration.Configuration, org.apache.flink.core.plugin.PluginManager): Argument leaf type org.apache.flink.core.plugin.PluginManager does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.core.fs.FileSystem.initialize(org.apache.flink.configuration.Configuration, org.apache.flink.core.plugin.PluginManager): Exception leaf type org.apache.flink.configuration.IllegalConfigurationException does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.core.io.InputSplitSource.getInputSplitAssigner([Lorg.apache.flink.core.io.InputSplit;): Returned leaf type org.apache.flink.core.io.InputSplitAssigner does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
+org.apache.flink.hadoopcompatibility.mapred.HadoopReducerWrappedFunction.apply(java.lang.Object, org.apache.flink.streaming.api.windowing.windows.GlobalWindow, java.lang.Iterable, org.apache.flink.util.Collector): Argument leaf type org.apache.flink.streaming.api.windowing.windows.GlobalWindow does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
+org.apache.flink.hadoopcompatibility.mapred.HadoopReducerWrappedFunction.apply(java.lang.Object, org.apache.flink.streaming.api.windowing.windows.Window, java.lang.Iterable, org.apache.flink.util.Collector): Argument leaf type org.apache.flink.streaming.api.windowing.windows.Window does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
+org.apache.flink.hadoopcompatibility.mapred.HadoopReducerWrappedFunction.apply(org.apache.flink.streaming.api.windowing.windows.GlobalWindow, java.lang.Iterable, org.apache.flink.util.Collector): Argument leaf type org.apache.flink.streaming.api.windowing.windows.GlobalWindow does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
+org.apache.flink.hadoopcompatibility.mapred.HadoopReducerWrappedFunction.apply(org.apache.flink.streaming.api.windowing.windows.Window, java.lang.Iterable, org.apache.flink.util.Collector): Argument leaf type org.apache.flink.streaming.api.windowing.windows.Window does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.runtime.jobgraph.tasks.InputSplitProvider.getNextInputSplit(java.lang.ClassLoader): Exception leaf type org.apache.flink.runtime.jobgraph.tasks.InputSplitProviderException does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.checkpoint.CheckpointedFunction.initializeState(org.apache.flink.runtime.state.FunctionInitializationContext): Argument leaf type org.apache.flink.runtime.state.FunctionInitializationContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.checkpoint.CheckpointedFunction.snapshotState(org.apache.flink.runtime.state.FunctionSnapshotContext): Argument leaf type org.apache.flink.runtime.state.FunctionSnapshotContext does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @Deprecated
diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8 b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
index ea4f71d6bba7e..56f0b028f9d39 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
+++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/5b9eed8a-5fb6-4373-98ac-3be2a71941b8
@@ -13,15 +13,6 @@ org.apache.flink.api.common.state.StateDescriptor.getType(): Returned leaf type
org.apache.flink.api.common.state.ValueStateDescriptor.getType(): Returned leaf type org.apache.flink.api.common.state.StateDescriptor$Type does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo.createComparator(boolean, org.apache.flink.api.common.ExecutionConfig): Returned leaf type org.apache.flink.api.common.typeutils.base.array.PrimitiveArrayComparator does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.api.common.typeutils.TypeComparator.putNormalizedKey(java.lang.Object, org.apache.flink.core.memory.MemorySegment, int, int): Argument leaf type org.apache.flink.core.memory.MemorySegment does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.io.SplitDataProperties.getSplitOrder(): Returned leaf type org.apache.flink.api.common.operators.Ordering does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.DataSink.getMinResources(): Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.DataSink.getPreferredResources(): Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.DeltaIteration.getAggregators(): Returned leaf type org.apache.flink.api.common.aggregators.AggregatorRegistry does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.DeltaIteration.getMinResources(): Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.DeltaIteration.getPreferredResources(): Returned leaf type org.apache.flink.api.common.operators.ResourceSpec does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.DistinctOperator.setCombineHint(org.apache.flink.api.common.operators.base.ReduceOperatorBase$CombineHint): Argument leaf type org.apache.flink.api.common.operators.base.ReduceOperatorBase$CombineHint does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.IterativeDataSet.getAggregators(): Returned leaf type org.apache.flink.api.common.aggregators.AggregatorRegistry does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
-org.apache.flink.api.java.operators.ReduceOperator.setCombineHint(org.apache.flink.api.common.operators.base.ReduceOperatorBase$CombineHint): Argument leaf type org.apache.flink.api.common.operators.base.ReduceOperatorBase$CombineHint does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.api.java.typeutils.PojoTypeInfo.getPojoFieldAt(int): Returned leaf type org.apache.flink.api.java.typeutils.PojoField does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.api.java.typeutils.TupleTypeInfo.createSerializer(org.apache.flink.api.common.serialization.SerializerConfig): Returned leaf type org.apache.flink.api.java.typeutils.runtime.TupleSerializer does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
org.apache.flink.cep.functions.PatternProcessFunction.processMatch(java.util.Map, org.apache.flink.cep.functions.PatternProcessFunction$Context, org.apache.flink.util.Collector): Argument leaf type org.apache.flink.cep.functions.PatternProcessFunction$Context does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated
diff --git a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/7602816f-5c01-4b7a-9e3e-235dfedec245 b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/7602816f-5c01-4b7a-9e3e-235dfedec245
index 7ecef56674d1d..ad569a3564e82 100644
--- a/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/7602816f-5c01-4b7a-9e3e-235dfedec245
+++ b/flink-architecture-tests/flink-architecture-tests-production/archunit-violations/7602816f-5c01-4b7a-9e3e-235dfedec245
@@ -86,48 +86,6 @@ org.apache.flink.api.common.typeutils.base.array.LongPrimitiveArraySerializer$Lo
org.apache.flink.api.common.typeutils.base.array.ShortPrimitiveArraySerializer$ShortPrimitiveArraySerializerSnapshot does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.common.typeutils.base.array.StringArraySerializer$StringArraySerializerSnapshot does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.connector.source.lib.NumberSequenceSource$NumberSequenceSplit does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.Utils$ChecksumHashCode does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.Utils$ChecksumHashCodeHelper does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.Utils$CollectHelper does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.Utils$CountHelper does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.aggregation.MaxAggregationFunction$MaxAggregationFunctionFactory does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.aggregation.MinAggregationFunction$MinAggregationFunctionFactory does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.aggregation.SumAggregationFunction$SumAggregationFunctionFactory does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFields does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsFirst does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.functions.FunctionAnnotation$ForwardedFieldsSecond does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFields does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsFirst does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.functions.FunctionAnnotation$NonForwardedFieldsSecond does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.io.SplitDataProperties$SourcePartitionerMarker does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.io.TextOutputFormat$TextFormatter does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSets does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSets$CoGroupOperatorSetsPredicate does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.CoGroupOperator$CoGroupOperatorSets$CoGroupOperatorSetsPredicate$CoGroupOperatorWithoutFunction does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.join.JoinOperatorSetsBase$JoinOperatorSetsPredicateBase does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.translation.PlanFilterOperator$FlatMapFilter does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.operators.translation.RichCombineToGroupCombineWrapper does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.DoubleSummaryAggregator$MaxDoubleAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.DoubleSummaryAggregator$MinDoubleAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.DoubleSummaryAggregator$SumDoubleAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.FloatSummaryAggregator$MaxFloatAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.FloatSummaryAggregator$MinFloatAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.FloatSummaryAggregator$SumFloatAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.IntegerSummaryAggregator$MaxIntegerAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.IntegerSummaryAggregator$MinIntegerAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.IntegerSummaryAggregator$SumIntegerAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.LongSummaryAggregator$MaxLongAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.LongSummaryAggregator$MinLongAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ShortSummaryAggregator$MaxShortAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ShortSummaryAggregator$MinShortAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ShortSummaryAggregator$SumShortAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$BooleanValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$DoubleValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$FloatValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$IntegerValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$LongValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$ShortValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.summarize.aggregation.ValueSummaryAggregator$StringValueSummaryAggregator does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.AvroUtils does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.EitherTypeInfoFactory does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.MissingTypeInfo does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
@@ -143,7 +101,6 @@ org.apache.flink.api.java.typeutils.runtime.RowSerializer$RowSerializerSnapshot
org.apache.flink.api.java.typeutils.runtime.Tuple0SerializerSnapshot does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.runtime.ValueSerializer$ValueSerializerSnapshot does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.runtime.kryo.ChillSerializerRegistrar does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
-org.apache.flink.api.java.typeutils.runtime.kryo.FlinkChillPackageRegistrar does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.runtime.kryo.JavaSerializer does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializerSnapshot does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
@@ -229,6 +186,7 @@ org.apache.flink.streaming.api.graph.StreamConfig$SourceInputConfig does not sat
org.apache.flink.streaming.api.graph.StreamGraphHasher does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.graph.StreamGraphHasherV2 does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.graph.StreamGraphUserHashHasher does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
+org.apache.flink.streaming.api.legacy.io.CollectionInputFormat does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.operators.BackendRestorerProcedure does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.operators.CountingOutput does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
org.apache.flink.streaming.api.operators.InputSelection$Builder does not satisfy: annotated with @Internal or annotated with @Experimental or annotated with @PublicEvolving or annotated with @Public or annotated with @Deprecated
diff --git a/flink-clients/pom.xml b/flink-clients/pom.xml
index 838de538771aa..a89ba2f7d0ade 100644
--- a/flink-clients/pom.xml
+++ b/flink-clients/pom.xml
@@ -57,18 +57,6 @@ under the License.
${project.version}
-
- org.apache.flink
- flink-optimizer
- ${project.version}
-
-
-
- org.apache.flink
- flink-java
- ${project.version}
-
-
commons-clicommons-cli
diff --git a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
index 88fcc98e3acb0..f8554ed3064a0 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/ClientUtils.java
@@ -21,7 +21,6 @@
import org.apache.flink.api.common.JobID;
import org.apache.flink.api.common.JobStatus;
import org.apache.flink.client.cli.ClientOptions;
-import org.apache.flink.client.program.ContextEnvironment;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.client.program.StreamContextEnvironment;
@@ -91,13 +90,6 @@ public static void executeProgram(
"Starting program (detached: {})",
!configuration.get(DeploymentOptions.ATTACHED));
- ContextEnvironment.setAsContext(
- executorServiceLoader,
- configuration,
- userCodeClassLoader,
- enforceSingleJobExecution,
- suppressSysout);
-
StreamContextEnvironment.setAsContext(
executorServiceLoader,
configuration,
@@ -112,7 +104,6 @@ public static void executeProgram(
try {
program.invokeInteractiveModeForExecution();
} finally {
- ContextEnvironment.unsetAsContext();
StreamContextEnvironment.unsetAsContext();
// For DataStream v2.
ExecutionContextEnvironment.unsetAsContext();
diff --git a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java
index 0aefcb3ef7989..80301861ab7b3 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/FlinkPipelineTranslationUtil.java
@@ -83,11 +83,6 @@ public static String translateToJSONExecutionPlan(
private static FlinkPipelineTranslator getPipelineTranslator(
ClassLoader userClassloader, Pipeline pipeline) {
- PlanTranslator planTranslator = new PlanTranslator();
-
- if (planTranslator.canTranslate(pipeline)) {
- return planTranslator;
- }
StreamGraphTranslator streamGraphTranslator = new StreamGraphTranslator(userClassloader);
diff --git a/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java b/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java
deleted file mode 100644
index 35287d80cdd0e..0000000000000
--- a/flink-clients/src/main/java/org/apache/flink/client/PlanTranslator.java
+++ /dev/null
@@ -1,94 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.client;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.dag.Pipeline;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
-import org.apache.flink.optimizer.plantranslate.JobGraphGenerator;
-import org.apache.flink.runtime.jobgraph.JobGraph;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.apache.flink.util.Preconditions.checkArgument;
-
-/** {@link FlinkPipelineTranslator} for DataSet API {@link Plan Plans}. */
-public class PlanTranslator implements FlinkPipelineTranslator {
-
- private static final Logger LOG = LoggerFactory.getLogger(PlanTranslator.class);
-
- @Override
- public JobGraph translateToJobGraph(
- Pipeline pipeline, Configuration optimizerConfiguration, int defaultParallelism) {
- checkArgument(pipeline instanceof Plan, "Given pipeline is not a DataSet Plan.");
-
- Plan plan = (Plan) pipeline;
- setDefaultParallelism(plan, defaultParallelism);
- return compilePlan(plan, optimizerConfiguration);
- }
-
- private void setDefaultParallelism(Plan plan, int defaultParallelism) {
- if (defaultParallelism > 0 && plan.getDefaultParallelism() <= 0) {
- LOG.debug(
- "Changing plan default parallelism from {} to {}",
- plan.getDefaultParallelism(),
- defaultParallelism);
- plan.setDefaultParallelism(defaultParallelism);
- }
-
- LOG.debug(
- "Set parallelism {}, plan default parallelism {}",
- defaultParallelism,
- plan.getDefaultParallelism());
- }
-
- @Override
- public String translateToJSONExecutionPlan(Pipeline pipeline) {
- checkArgument(pipeline instanceof Plan, "Given pipeline is not a DataSet Plan.");
-
- Plan plan = (Plan) pipeline;
-
- Optimizer opt =
- new Optimizer(
- new DataStatistics(), new DefaultCostEstimator(), new Configuration());
- OptimizedPlan optPlan = opt.compile(plan);
-
- return new PlanJSONDumpGenerator().getOptimizerPlanAsJSON(optPlan);
- }
-
- private JobGraph compilePlan(Plan plan, Configuration optimizerConfiguration) {
- Optimizer optimizer = new Optimizer(new DataStatistics(), optimizerConfiguration);
- OptimizedPlan optimizedPlan = optimizer.compile(plan);
-
- JobGraphGenerator jobGraphGenerator = new JobGraphGenerator(optimizerConfiguration);
- return jobGraphGenerator.compileJobGraph(optimizedPlan, plan.getJobId());
- }
-
- @Override
- public boolean canTranslate(Pipeline pipeline) {
- return pipeline instanceof Plan;
- }
-}
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
deleted file mode 100644
index fe4db252c1e0c..0000000000000
--- a/flink-clients/src/main/java/org/apache/flink/client/program/ContextEnvironment.java
+++ /dev/null
@@ -1,196 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.program;
-
-import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironmentFactory;
-import org.apache.flink.client.ClientUtils;
-import org.apache.flink.client.cli.ClientOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.DeploymentOptions;
-import org.apache.flink.core.execution.DetachedJobExecutionResult;
-import org.apache.flink.core.execution.JobClient;
-import org.apache.flink.core.execution.JobListener;
-import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
-import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.FlinkRuntimeException;
-import org.apache.flink.util.ShutdownHookUtil;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/** Execution Environment for remote execution with the Client. */
-public class ContextEnvironment extends ExecutionEnvironment {
-
- private static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class);
-
- private final boolean suppressSysout;
-
- private final boolean enforceSingleJobExecution;
-
- private int jobCounter;
-
- public ContextEnvironment(
- final PipelineExecutorServiceLoader executorServiceLoader,
- final Configuration configuration,
- final ClassLoader userCodeClassLoader,
- final boolean enforceSingleJobExecution,
- final boolean suppressSysout) {
- super(executorServiceLoader, configuration, userCodeClassLoader);
- this.suppressSysout = suppressSysout;
- this.enforceSingleJobExecution = enforceSingleJobExecution;
-
- this.jobCounter = 0;
- }
-
- @Override
- public JobExecutionResult execute(String jobName) throws Exception {
- final JobClient jobClient = executeAsync(jobName);
- final List jobListeners = getJobListeners();
-
- try {
- final JobExecutionResult jobExecutionResult = getJobExecutionResult(jobClient);
- jobListeners.forEach(
- jobListener -> jobListener.onJobExecuted(jobExecutionResult, null));
- return jobExecutionResult;
- } catch (Throwable t) {
- jobListeners.forEach(
- jobListener ->
- jobListener.onJobExecuted(
- null, ExceptionUtils.stripExecutionException(t)));
- ExceptionUtils.rethrowException(t);
-
- // never reached, only make javac happy
- return null;
- }
- }
-
- private JobExecutionResult getJobExecutionResult(final JobClient jobClient) throws Exception {
- checkNotNull(jobClient);
-
- JobExecutionResult jobExecutionResult;
- if (getConfiguration().get(DeploymentOptions.ATTACHED)) {
- CompletableFuture jobExecutionResultFuture =
- jobClient.getJobExecutionResult();
-
- ScheduledExecutorService clientHeartbeatService = null;
- if (getConfiguration().get(DeploymentOptions.SHUTDOWN_IF_ATTACHED)) {
- Thread shutdownHook =
- ShutdownHookUtil.addShutdownHook(
- () -> {
- // wait a smidgen to allow the async request to go through
- // before
- // the jvm exits
- jobClient.cancel().get(1, TimeUnit.SECONDS);
- },
- ContextEnvironment.class.getSimpleName(),
- LOG);
- jobExecutionResultFuture.whenComplete(
- (ignored, throwable) ->
- ShutdownHookUtil.removeShutdownHook(
- shutdownHook,
- ContextEnvironment.class.getSimpleName(),
- LOG));
- clientHeartbeatService =
- ClientUtils.reportHeartbeatPeriodically(
- jobClient,
- getConfiguration()
- .get(ClientOptions.CLIENT_HEARTBEAT_INTERVAL)
- .toMillis(),
- getConfiguration()
- .get(ClientOptions.CLIENT_HEARTBEAT_TIMEOUT)
- .toMillis());
- }
-
- jobExecutionResult = jobExecutionResultFuture.get();
- if (clientHeartbeatService != null) {
- clientHeartbeatService.shutdown();
- }
- if (!suppressSysout) {
- System.out.println(jobExecutionResult);
- }
- LOG.info(String.valueOf(jobExecutionResult));
- } else {
- jobExecutionResult = new DetachedJobExecutionResult(jobClient.getJobID());
- }
-
- return jobExecutionResult;
- }
-
- @Override
- public JobClient executeAsync(String jobName) throws Exception {
- validateAllowedExecution();
- final JobClient jobClient = super.executeAsync(jobName);
-
- if (!suppressSysout) {
- System.out.println("Job has been submitted with JobID " + jobClient.getJobID());
- }
- LOG.info("Job has been submitted with JobID {}", jobClient.getJobID());
- return jobClient;
- }
-
- private void validateAllowedExecution() {
- if (enforceSingleJobExecution && jobCounter > 0) {
- throw new FlinkRuntimeException(
- "Cannot have more than one execute() or executeAsync() call in a single environment.");
- }
- jobCounter++;
- }
-
- @Override
- public String toString() {
- return "Context Environment (parallelism = "
- + (getParallelism() == ExecutionConfig.PARALLELISM_DEFAULT
- ? "default"
- : getParallelism())
- + ")";
- }
-
- // --------------------------------------------------------------------------------------------
-
- public static void setAsContext(
- final PipelineExecutorServiceLoader executorServiceLoader,
- final Configuration configuration,
- final ClassLoader userCodeClassLoader,
- final boolean enforceSingleJobExecution,
- final boolean suppressSysout) {
- ExecutionEnvironmentFactory factory =
- () ->
- new ContextEnvironment(
- executorServiceLoader,
- configuration,
- userCodeClassLoader,
- enforceSingleJobExecution,
- suppressSysout);
- initializeContextEnvironment(factory);
- }
-
- public static void unsetAsContext() {
- resetContextEnvironment();
- }
-}
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
deleted file mode 100644
index 0db27afa6a30d..0000000000000
--- a/flink-clients/src/main/java/org/apache/flink/client/program/OptimizerPlanEnvironment.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.program;
-
-import org.apache.flink.api.dag.Pipeline;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.ExecutionEnvironmentFactory;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.execution.JobClient;
-
-/**
- * An {@link ExecutionEnvironment} that never executes a job but only extracts the {@link Pipeline}.
- */
-public class OptimizerPlanEnvironment extends ExecutionEnvironment {
-
- private Pipeline pipeline;
-
- public Pipeline getPipeline() {
- return pipeline;
- }
-
- public OptimizerPlanEnvironment(
- Configuration configuration, ClassLoader userClassloader, int parallelism) {
- super(configuration, userClassloader);
- if (parallelism > 0) {
- setParallelism(parallelism);
- }
- }
-
- @Override
- public JobClient executeAsync(String jobName) {
- pipeline = createProgramPlan();
-
- // do not go on with anything now!
- throw new ProgramAbortException();
- }
-
- public void setAsContext() {
- ExecutionEnvironmentFactory factory = () -> this;
- initializeContextEnvironment(factory);
- }
-
- public void unsetAsContext() {
- resetContextEnvironment();
- }
-}
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java
index 8659a0b6eb8aa..6f105e192156c 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/PackagedProgramUtils.java
@@ -23,7 +23,6 @@
import org.apache.flink.client.FlinkPipelineTranslationUtil;
import org.apache.flink.configuration.ConfigConstants;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.CompilerException;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.commons.collections.CollectionUtils;
@@ -123,7 +122,7 @@ public static Pipeline getPipelineFromProgram(
Configuration configuration,
int parallelism,
boolean suppressOutput)
- throws CompilerException, ProgramInvocationException {
+ throws ProgramInvocationException {
final ClassLoader contextClassLoader = Thread.currentThread().getContextClassLoader();
Thread.currentThread().setContextClassLoader(program.getUserCodeClassLoader());
@@ -145,10 +144,6 @@ public static Pipeline getPipelineFromProgram(
}
// temporary hack to support the optimizer plan preview
- OptimizerPlanEnvironment benv =
- new OptimizerPlanEnvironment(
- configuration, program.getUserCodeClassLoader(), parallelism);
- benv.setAsContext();
StreamPlanEnvironment senv =
new StreamPlanEnvironment(
configuration, program.getUserCodeClassLoader(), parallelism);
@@ -157,10 +152,6 @@ public static Pipeline getPipelineFromProgram(
try {
program.invokeInteractiveModeForExecution();
} catch (Throwable t) {
- if (benv.getPipeline() != null) {
- return benv.getPipeline();
- }
-
if (senv.getPipeline() != null) {
return senv.getPipeline();
}
@@ -172,7 +163,6 @@ public static Pipeline getPipelineFromProgram(
throw generateException(
program, "The program caused an error: ", t, stdOutBuffer, stdErrBuffer);
} finally {
- benv.unsetAsContext();
senv.unsetAsContext();
if (suppressOutput) {
System.setOut(originalOut);
diff --git a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
index 11c8ba1fdf7ec..a615c23b5e471 100644
--- a/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
+++ b/flink-clients/src/main/java/org/apache/flink/client/program/StreamContextEnvironment.java
@@ -20,7 +20,6 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.annotation.PublicEvolving;
import org.apache.flink.api.common.JobExecutionResult;
-import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.cli.ClientOptions;
import org.apache.flink.configuration.Configuration;
@@ -61,7 +60,7 @@
@PublicEvolving
public class StreamContextEnvironment extends StreamExecutionEnvironment {
- private static final Logger LOG = LoggerFactory.getLogger(ExecutionEnvironment.class);
+ private static final Logger LOG = LoggerFactory.getLogger(StreamContextEnvironment.class);
private final boolean suppressSysout;
diff --git a/flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.java b/flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.java
deleted file mode 100644
index b853cdda73b2f..0000000000000
--- a/flink-clients/src/test/java/org/apache/flink/client/ExecutionEnvironmentTest.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- *
- */
-
-package org.apache.flink.client;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-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.configuration.PipelineOptions;
-
-import org.junit.jupiter.api.Test;
-
-import java.io.Serializable;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/**
- * Tests for {@link ExecutionEnvironment}.
- *
- *
NOTE: This test is in the flink-client package because we cannot have it in flink-java, where
- * the JSON plan generator is not available. Making it available, by depending on flink-optimizer
- * would create a cyclic dependency.
- */
-class ExecutionEnvironmentTest implements Serializable {
-
- /**
- * Tests that verifies consecutive calls to {@link ExecutionEnvironment#getExecutionPlan()} do
- * not cause any exceptions. {@link ExecutionEnvironment#getExecutionPlan()} must not modify the
- * state of the plan
- */
- @Test
- void testExecuteAfterGetExecutionPlanContextEnvironment() {
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- DataSet baseSet = env.fromElements(1, 2);
-
- DataSet result = baseSet.map((MapFunction) value -> value * 2);
- result.output(new DiscardingOutputFormat<>());
-
- try {
- env.getExecutionPlan();
- env.getExecutionPlan();
- } catch (Exception e) {
- fail("Consecutive #getExecutionPlan calls caused an exception.");
- }
- }
-
- @Test
- void testDefaultJobName() {
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- testJobName("Flink Java Job at", env);
- }
-
- @Test
- void testUserDefinedJobName() {
- String jobName = "MyTestJob";
- Configuration config = new Configuration();
- config.set(PipelineOptions.NAME, jobName);
- ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment(config);
- testJobName(jobName, env);
- }
-
- @Test
- void testUserDefinedJobNameWithConfigure() {
- String jobName = "MyTestJob";
- Configuration config = new Configuration();
- config.set(PipelineOptions.NAME, jobName);
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.configure(config, this.getClass().getClassLoader());
- testJobName(jobName, env);
- }
-
- private void testJobName(String prefixOfExpectedJobName, ExecutionEnvironment env) {
- env.fromElements(1, 2, 3).writeAsText("/dev/null");
- Plan plan = env.createProgramPlan();
- assertThat(plan.getJobName()).startsWith(prefixOfExpectedJobName);
- }
-}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
index 82f95cc08c38d..5cd7efceca4e7 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/cli/CliFrontendPackageProgramTest.java
@@ -24,9 +24,6 @@
import org.apache.flink.client.program.PackagedProgramUtils;
import org.apache.flink.client.program.ProgramInvocationException;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
import org.apache.flink.runtime.jobgraph.SavepointRestoreSettings;
import org.apache.commons.cli.CommandLine;
@@ -322,7 +319,6 @@ public Class> loadClass(String name) throws ClassNotFoundException {
assertThat(prog.getArguments()).isEqualTo(reducedArguments);
Configuration c = new Configuration();
- Optimizer compiler = new Optimizer(new DataStatistics(), new DefaultCostEstimator(), c);
// we expect this to fail with a "ClassNotFoundException"
Pipeline pipeline = PackagedProgramUtils.getPipelineFromProgram(prog, c, 666, true);
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
index 7b53a79a147d5..4b6c092961281 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/ClientTest.java
@@ -21,13 +21,9 @@
import org.apache.flink.api.common.InvalidProgramException;
import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.JobID;
-import org.apache.flink.api.common.Plan;
import org.apache.flink.api.common.ProgramDescription;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.api.dag.Pipeline;
import org.apache.flink.client.ClientUtils;
import org.apache.flink.client.FlinkPipelineTranslationUtil;
import org.apache.flink.client.cli.ExecutionConfigAccessor;
@@ -45,14 +41,13 @@
import org.apache.flink.core.execution.PipelineExecutor;
import org.apache.flink.core.execution.PipelineExecutorFactory;
import org.apache.flink.core.execution.PipelineExecutorServiceLoader;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
import org.apache.flink.runtime.jobgraph.JobGraph;
import org.apache.flink.runtime.testutils.InternalMiniClusterExtension;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
+import org.apache.flink.streaming.api.graph.StreamGraph;
import org.apache.flink.util.FlinkRuntimeException;
import org.junit.jupiter.api.BeforeEach;
@@ -80,7 +75,7 @@ class ClientTest {
new InternalMiniClusterExtension(
new MiniClusterResourceConfiguration.Builder().build());
- private Plan plan;
+ private StreamGraph streamGraph;
private Configuration config;
@@ -94,9 +89,9 @@ class ClientTest {
@BeforeEach
void setUp() {
- ExecutionEnvironment env = ExecutionEnvironment.createLocalEnvironment();
- env.generateSequence(1, 1000).output(new DiscardingOutputFormat<>());
- plan = env.createProgramPlan();
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment();
+ env.fromSequence(1, 1000).sinkTo(new DiscardingSink<>());
+ streamGraph = env.getStreamGraph();
config = new Configuration();
config.set(JobManagerOptions.ADDRESS, "localhost");
@@ -135,7 +130,7 @@ void testDetachedMode() {
final Configuration configuration = fromPackagedProgram(prg, 1, true);
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(clusterClient, plan),
+ new TestExecutorServiceLoader(clusterClient, streamGraph),
configuration,
prg,
false,
@@ -157,7 +152,7 @@ void testDetachedMode() {
final Configuration configuration = fromPackagedProgram(prg, 1, true);
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(clusterClient, plan),
+ new TestExecutorServiceLoader(clusterClient, streamGraph),
configuration,
prg,
false,
@@ -179,7 +174,7 @@ void testDetachedMode() {
final Configuration configuration = fromPackagedProgram(prg, 1, true);
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(clusterClient, plan),
+ new TestExecutorServiceLoader(clusterClient, streamGraph),
configuration,
prg,
false,
@@ -202,7 +197,7 @@ void testDetachedMode() {
final Configuration configuration = fromPackagedProgram(prg, 1, true);
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(clusterClient, plan),
+ new TestExecutorServiceLoader(clusterClient, streamGraph),
configuration,
prg,
false,
@@ -250,7 +245,7 @@ private void launchMultiExecuteJob(final boolean enforceSingleJobExecution)
final Configuration configuration = fromPackagedProgram(program, 1, false);
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(clusterClient, plan),
+ new TestExecutorServiceLoader(clusterClient, streamGraph),
configuration,
program,
enforceSingleJobExecution,
@@ -263,12 +258,7 @@ private void launchMultiExecuteJob(final boolean enforceSingleJobExecution)
void shouldSubmitToJobClient() {
final ClusterClient> clusterClient =
new MiniClusterClient(new Configuration(), MINI_CLUSTER_RESOURCE.getMiniCluster());
- JobGraph jobGraph =
- FlinkPipelineTranslationUtil.getJobGraph(
- Thread.currentThread().getContextClassLoader(),
- plan,
- new Configuration(),
- 1);
+ JobGraph jobGraph = streamGraph.getJobGraph();
jobGraph.addJars(Collections.emptyList());
jobGraph.setClasspaths(Collections.emptyList());
@@ -278,7 +268,7 @@ void shouldSubmitToJobClient() {
public static class TestEntrypoint {
public static void main(String[] args) {
- ExecutionEnvironment.createLocalEnvironment();
+ StreamExecutionEnvironment.createLocalEnvironment();
}
}
@@ -301,7 +291,7 @@ void tryLocalExecution() throws ProgramInvocationException {
final Configuration configuration =
fromPackagedProgram(packagedProgramMock, 1, true);
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(client, plan),
+ new TestExecutorServiceLoader(client, streamGraph),
configuration,
packagedProgramMock,
false,
@@ -318,28 +308,24 @@ void tryLocalExecution() throws ProgramInvocationException {
void testGetExecutionPlan() throws ProgramInvocationException {
PackagedProgram prg =
PackagedProgram.newBuilder()
- .setEntryPointClassName(TestOptimizerPlan.class.getName())
+ .setEntryPointClassName(TestExecutionPlan.class.getName())
.setArguments("/dev/random", "/tmp")
.build();
- Optimizer optimizer =
- new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
- Plan plan =
- (Plan)
- PackagedProgramUtils.getPipelineFromProgram(
- prg, new Configuration(), 1, true);
- OptimizedPlan op = optimizer.compile(plan);
- assertThat(op).isNotNull();
-
- PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();
- assertThat(dumper.getOptimizerPlanAsJSON(op)).isNotNull();
-
- // test HTML escaping
- PlanJSONDumpGenerator dumper2 = new PlanJSONDumpGenerator();
- dumper2.setEncodeForHTML(true);
- String htmlEscaped = dumper2.getOptimizerPlanAsJSON(op);
-
- assertThat(htmlEscaped).doesNotContain("\\");
+ Pipeline pipeline =
+ PackagedProgramUtils.getPipelineFromProgram(prg, new Configuration(), 666, true);
+ String jsonExecutionPlan =
+ FlinkPipelineTranslationUtil.translateToJSONExecutionPlan(
+ prg.getUserCodeClassLoader(), pipeline);
+ assertThat(jsonExecutionPlan).isNotNull();
+ assertThat(jsonExecutionPlan)
+ .contains(
+ "\"type\" : \"Source: MySource\"",
+ "\"type\" : \"MyMap\",",
+ "\"type\" : \"MySink: Writer\",");
+ assertThat(jsonExecutionPlan).contains("\"parallelism\" : 666");
+ assertThat(jsonExecutionPlan).doesNotContain("\\");
+ System.out.println(jsonExecutionPlan);
}
@Test
@@ -359,7 +345,8 @@ void testFailOnForbiddenConfiguration() throws ProgramInvocationException {
assertThatThrownBy(
() ->
ClientUtils.executeProgram(
- new TestExecutorServiceLoader(clusterClient, plan),
+ new TestExecutorServiceLoader(
+ clusterClient, streamGraph),
configuration,
program,
true,
@@ -371,34 +358,38 @@ void testFailOnForbiddenConfiguration() throws ProgramInvocationException {
// --------------------------------------------------------------------------------------------
/** A test job. */
- public static class TestOptimizerPlan implements ProgramDescription {
+ public static class TestExecutionPlan implements ProgramDescription {
@SuppressWarnings("serial")
public static void main(String[] args) throws Exception {
if (args.length < 2) {
- System.err.println("Usage: TestOptimizerPlan ");
+ System.err.println("Usage: TestExecutionPlan");
return;
}
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataSet> input =
- env.readCsvFile(args[0]).fieldDelimiter("\t").types(Long.class, Long.class);
+ DataStream input = env.fromSequence(0, 999).name("MySource");
- DataSet> result =
+ DataStream result =
input.map(
- new MapFunction, Tuple2>() {
- public Tuple2 map(Tuple2 value) {
- return new Tuple2(value.f0, value.f1 + 1);
- }
- });
- result.writeAsCsv(args[1], "\n", "\t");
+ new MapFunction() {
+
+ @Override
+ public Long map(Long value) throws Exception {
+ return value * 2 + 1;
+ }
+ })
+ .name("MyMap");
+
+ result.sinkTo(new DiscardingSink<>()).name("MySink");
+
env.execute();
}
@Override
public String getDescription() {
- return "TestOptimizerPlan ";
+ return "TestExecutionPlan";
}
}
@@ -406,8 +397,11 @@ public String getDescription() {
public static final class TestEager {
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromElements(1, 2).collect();
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(1, 2).sinkTo(new DiscardingSink<>());
+ JobExecutionResult result = env.execute();
+ result.getAccumulatorResult("dummy");
}
}
@@ -415,10 +409,11 @@ public static void main(String[] args) throws Exception {
public static final class TestMultiExecute {
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
for (int i = 0; i < 2; i++) {
- env.fromElements(1, 2).output(new DiscardingOutputFormat<>());
+ env.fromData(1, 2).sinkTo(new DiscardingSink<>());
JobClient jc = env.executeAsync();
jc.getJobExecutionResult();
@@ -430,8 +425,9 @@ public static void main(String[] args) throws Exception {
public static final class TestGetRuntime {
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromElements(1, 2).output(new DiscardingOutputFormat());
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(1, 2).sinkTo(new DiscardingSink<>());
env.execute().getNetRuntime();
}
}
@@ -440,8 +436,9 @@ public static void main(String[] args) throws Exception {
public static final class TestGetJobID {
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromElements(1, 2).output(new DiscardingOutputFormat());
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(1, 2).sinkTo(new DiscardingSink<>());
env.execute().getJobID();
}
}
@@ -450,8 +447,9 @@ public static void main(String[] args) throws Exception {
public static final class TestGetAccumulator {
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromElements(1, 2).output(new DiscardingOutputFormat());
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(1, 2).sinkTo(new DiscardingSink<>());
env.execute().getAccumulatorResult(ACCUMULATOR_NAME);
}
}
@@ -460,8 +458,9 @@ public static void main(String[] args) throws Exception {
public static final class TestGetAllAccumulator {
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromElements(1, 2).output(new DiscardingOutputFormat());
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(1, 2).sinkTo(new DiscardingSink<>());
env.execute().getAllAccumulatorResults();
}
}
@@ -470,11 +469,12 @@ private static final class TestExecutorServiceLoader implements PipelineExecutor
private final ClusterClient> clusterClient;
- private final Plan plan;
+ private final StreamGraph streamGraph;
- TestExecutorServiceLoader(final ClusterClient> clusterClient, final Plan plan) {
+ TestExecutorServiceLoader(
+ final ClusterClient> clusterClient, final StreamGraph streamGraph) {
this.clusterClient = checkNotNull(clusterClient);
- this.plan = checkNotNull(plan);
+ this.streamGraph = checkNotNull(streamGraph);
}
@Override
@@ -496,9 +496,7 @@ public boolean isCompatibleWith(@Nonnull Configuration configuration) {
public PipelineExecutor getExecutor(@Nonnull Configuration configuration) {
return (pipeline, config, classLoader) -> {
final int parallelism = config.get(CoreOptions.DEFAULT_PARALLELISM);
- final JobGraph jobGraph =
- FlinkPipelineTranslationUtil.getJobGraph(
- classLoader, plan, config, parallelism);
+ final JobGraph jobGraph = streamGraph.getJobGraph();
final ExecutionConfigAccessor accessor =
ExecutionConfigAccessor.fromConfiguration(config);
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java
deleted file mode 100644
index c5683912d70e3..0000000000000
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanAfterExecutionTest.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.program;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.LocalEnvironment;
-import org.apache.flink.api.java.aggregation.Aggregations;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
-import org.apache.flink.api.java.tuple.Tuple2;
-
-import org.junit.jupiter.api.Test;
-
-import java.io.Serializable;
-
-import static org.assertj.core.api.Assertions.fail;
-
-/**
- * Tests that verify subsequent calls to {@link ExecutionEnvironment#getExecutionPlan()} and {@link
- * ExecutionEnvironment#execute()}/{@link ExecutionEnvironment#createProgramPlan()} do not cause any
- * exceptions.
- */
-@SuppressWarnings("serial")
-class ExecutionPlanAfterExecutionTest implements Serializable {
-
- @Test
- void testExecuteAfterGetExecutionPlan() {
- ExecutionEnvironment env = new LocalEnvironment();
-
- DataSet baseSet = env.fromElements(1, 2);
-
- DataSet result =
- baseSet.map(
- new MapFunction() {
- @Override
- public Integer map(Integer value) throws Exception {
- return value * 2;
- }
- });
- result.output(new DiscardingOutputFormat());
-
- try {
- env.getExecutionPlan();
- env.execute();
- } catch (Exception e) {
- e.printStackTrace();
- fail("Cannot run both #getExecutionPlan and #execute.");
- }
- }
-
- @Test
- void testCreatePlanAfterGetExecutionPlan() {
- ExecutionEnvironment env = new LocalEnvironment();
-
- DataSet baseSet = env.fromElements(1, 2);
-
- DataSet result = baseSet.map((MapFunction) value -> value * 2);
- result.output(new DiscardingOutputFormat());
-
- try {
- env.getExecutionPlan();
- env.createProgramPlan();
- } catch (Exception e) {
- e.printStackTrace();
- fail("Cannot run both #getExecutionPlan and #execute. Message: " + e.getMessage());
- }
- }
-
- @Test
- void testGetExecutionPlanOfRangePartition() {
- ExecutionEnvironment env = new LocalEnvironment();
-
- DataSet baseSet = env.fromElements(1, 2);
-
- DataSet> result =
- baseSet.map(
- new MapFunction>() {
- @Override
- public Tuple2 map(Integer value)
- throws Exception {
- return new Tuple2<>(value, value * 2);
- }
- })
- .partitionByRange(0)
- .aggregate(Aggregations.MAX, 1);
- result.output(new DiscardingOutputFormat>());
-
- try {
- env.getExecutionPlan();
- env.execute();
- } catch (Exception e) {
- e.printStackTrace();
- fail("Cannot run both #getExecutionPlan and #execute.");
- }
- }
-}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
deleted file mode 100644
index 6d2e6fb842ed2..0000000000000
--- a/flink-clients/src/test/java/org/apache/flink/client/program/ExecutionPlanCreationTest.java
+++ /dev/null
@@ -1,115 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.program;
-
-import org.apache.flink.api.common.Plan;
-import org.apache.flink.api.common.ProgramDescription;
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.JobManagerOptions;
-import org.apache.flink.optimizer.DataStatistics;
-import org.apache.flink.optimizer.Optimizer;
-import org.apache.flink.optimizer.costs.DefaultCostEstimator;
-import org.apache.flink.optimizer.plan.OptimizedPlan;
-import org.apache.flink.optimizer.plandump.PlanJSONDumpGenerator;
-
-import org.junit.jupiter.api.Test;
-
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Tests for the generation of execution plans. */
-class ExecutionPlanCreationTest {
-
- @Test
- void testGetExecutionPlan() {
- try {
- PackagedProgram prg =
- PackagedProgram.newBuilder()
- .setEntryPointClassName(TestOptimizerPlan.class.getName())
- .setArguments("/dev/random", "/tmp")
- .build();
-
- InetAddress mockAddress = InetAddress.getLocalHost();
- InetSocketAddress mockJmAddress = new InetSocketAddress(mockAddress, 12345);
-
- Configuration config = new Configuration();
-
- config.set(JobManagerOptions.ADDRESS, mockJmAddress.getHostName());
- config.set(JobManagerOptions.PORT, mockJmAddress.getPort());
-
- Optimizer optimizer =
- new Optimizer(new DataStatistics(), new DefaultCostEstimator(), config);
- Plan plan = (Plan) PackagedProgramUtils.getPipelineFromProgram(prg, config, -1, true);
- OptimizedPlan op = optimizer.compile(plan);
- assertThat(op).isNotNull();
-
- PlanJSONDumpGenerator dumper = new PlanJSONDumpGenerator();
- assertThat(dumper.getOptimizerPlanAsJSON(op)).isNotNull();
-
- // test HTML escaping
- PlanJSONDumpGenerator dumper2 = new PlanJSONDumpGenerator();
- dumper2.setEncodeForHTML(true);
- String htmlEscaped = dumper2.getOptimizerPlanAsJSON(op);
-
- assertThat(htmlEscaped).doesNotContain("\\");
- } catch (Exception e) {
- e.printStackTrace();
- fail(e.getMessage());
- }
- }
-
- /** A test job. */
- public static class TestOptimizerPlan implements ProgramDescription {
-
- @SuppressWarnings("serial")
- public static void main(String[] args) throws Exception {
- if (args.length < 2) {
- System.err.println("Usage: TestOptimizerPlan ");
- return;
- }
-
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- DataSet> input =
- env.readCsvFile(args[0]).fieldDelimiter("\t").types(Long.class, Long.class);
-
- DataSet> result =
- input.map(
- new MapFunction, Tuple2>() {
- public Tuple2 map(Tuple2 value) {
- return new Tuple2(value.f0, value.f1 + 1);
- }
- });
- result.writeAsCsv(args[1], "\n", "\t");
- env.execute();
- }
-
- @Override
- public String getDescription() {
- return "TestOptimizerPlan ";
- }
- }
-}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java
deleted file mode 100644
index c3c4184eb8ad1..0000000000000
--- a/flink-clients/src/test/java/org/apache/flink/client/program/OptimizerPlanEnvironmentTest.java
+++ /dev/null
@@ -1,68 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.client.program;
-
-import org.apache.flink.configuration.Configuration;
-
-import org.junit.jupiter.api.Test;
-
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-import static org.assertj.core.api.Assertions.fail;
-
-/** Tests for {@link OptimizerPlanEnvironment}. */
-public class OptimizerPlanEnvironmentTest {
-
- /**
- * Test the two modes for handling stdout/stderr of user program. (1) Capturing the output and
- * including it only in the exception (2) Leaving the output untouched
- */
- @Test
- void testStdOutStdErrHandling() throws Exception {
- runOutputTest(true, new String[] {"System.out: hello out!", "System.err: hello err!"});
- runOutputTest(false, new String[] {"System.out: (none)", "System.err: (none)"});
- }
-
- private void runOutputTest(boolean suppressOutput, String[] expectedCapturedOutput)
- throws ProgramInvocationException {
- PackagedProgram packagedProgram =
- PackagedProgram.newBuilder().setEntryPointClassName(getClass().getName()).build();
-
- assertThatThrownBy(
- () -> {
- // Flink will throw an error because no job graph will be generated by
- // the main method.
- PackagedProgramUtils.getPipelineFromProgram(
- packagedProgram, new Configuration(), 1, suppressOutput);
- fail("This should have failed to create the Flink Plan.");
- })
- .isInstanceOf(ProgramInvocationException.class)
- .hasMessageContainingAll(expectedCapturedOutput);
- }
-
- /**
- * Main method for {@code testEnsureStdoutStdErrIsRestored()}. This will not create a valid
- * Flink program. We will just use this program to check whether stdout/stderr is captured in a
- * byte buffer or directly printed to the console.
- */
- public static void main(String[] args) {
- // Print something to stdout/stderr for output suppression test
- System.out.println("hello out!");
- System.err.println("hello err!");
- }
-}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsPipelineTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsPipelineTest.java
index 9cbcdc1ba7517..4f2a31e960211 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsPipelineTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsPipelineTest.java
@@ -19,9 +19,7 @@
package org.apache.flink.client.program;
import org.apache.flink.api.common.ExecutionConfig;
-import org.apache.flink.api.common.Plan;
import org.apache.flink.api.dag.Pipeline;
-import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.PipelineOptions;
import org.apache.flink.runtime.execution.Environment;
@@ -62,9 +60,6 @@ public class PackagedProgramUtilsPipelineTest {
@Parameters(name = "testParameter-{0}")
public static Collection parameters() {
return Arrays.asList(
- TestParameter.of(
- DataSetTestProgram.class,
- pipeline -> ((Plan) pipeline).getExecutionConfig()),
TestParameter.of(
DataStreamTestProgram.class,
pipeline -> ((StreamGraph) pipeline).getExecutionConfig()));
@@ -78,9 +73,9 @@ public static Collection parameters() {
void testConfigurationForwarding() throws Exception {
// we want to test forwarding with this config, ensure that the default is what we expect.
assertThat(
- ExecutionEnvironment.getExecutionEnvironment()
+ StreamExecutionEnvironment.getExecutionEnvironment()
.getConfig()
- .isAutoTypeRegistrationDisabled())
+ .isObjectReuseEnabled())
.isFalse();
PackagedProgram packagedProgram =
@@ -89,7 +84,7 @@ void testConfigurationForwarding() throws Exception {
.build();
Configuration config = new Configuration();
- config.set(PipelineOptions.AUTO_TYPE_REGISTRATION, false);
+ config.set(PipelineOptions.OBJECT_REUSE, true);
Pipeline pipeline =
PackagedProgramUtils.getPipelineFromProgram(
@@ -98,7 +93,7 @@ void testConfigurationForwarding() throws Exception {
ExecutionConfig executionConfig = testParameter.extractExecutionConfig(pipeline);
// we want to test forwarding with this config, ensure that the default is what we expect.
- assertThat(executionConfig.isAutoTypeRegistrationDisabled()).isTrue();
+ assertThat(executionConfig.isObjectReuseEnabled()).isTrue();
}
@TestTemplate
@@ -189,15 +184,6 @@ public String toString() {
}
}
- /** Test Program for the DataSet API. */
- public static class DataSetTestProgram {
- public static void main(String[] args) throws Exception {
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromElements("hello").print();
- env.execute();
- }
- }
-
/** Test Program for the DataStream API. */
public static class DataStreamTestProgram {
public static void main(String[] args) throws Exception {
diff --git a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java
index 939e96edf77e2..0acea3c7fb343 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/program/PackagedProgramUtilsTest.java
@@ -18,7 +18,6 @@
package org.apache.flink.client.program;
-import org.apache.flink.api.java.DataSet;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.junit.jupiter.api.Test;
@@ -33,7 +32,7 @@
* Tests {@link PackagedProgramUtils}.
*
*
See also {@link PackagedProgramUtilsPipelineTest} for tests that need to test behaviour of
- * {@link DataStream} and {@link DataSet} programs.
+ * {@link DataStream} programs.
*/
class PackagedProgramUtilsTest {
diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/BlockingJob.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/BlockingJob.java
index 69fcfb52f7b9d..2c6ad21d16630 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/testjar/BlockingJob.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/BlockingJob.java
@@ -18,17 +18,16 @@
package org.apache.flink.client.testjar;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.client.cli.CliFrontendTestUtils;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
import org.apache.flink.util.FlinkException;
import java.io.File;
import java.io.FileNotFoundException;
import java.net.MalformedURLException;
-import java.util.Arrays;
import java.util.Collections;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
@@ -73,8 +72,8 @@ public static void unblock(String blockId) {
public static void main(String[] args) throws Exception {
final String blockId = args[0];
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromCollection(Arrays.asList(1, 2, 3))
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(1, 2, 3)
.map(element -> element + 1)
.map(
element -> {
@@ -84,7 +83,7 @@ public static void main(String[] args) throws Exception {
.await();
return element;
})
- .output(new DiscardingOutputFormat<>());
+ .sinkTo(new DiscardingSink<>());
env.execute();
}
}
diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java
index 83492c257eea2..0e07a721844df 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/ErrorHandlingSubmissionJob.java
@@ -18,11 +18,11 @@
package org.apache.flink.client.testjar;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.client.cli.CliFrontendTestUtils;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
import org.apache.flink.util.FlinkException;
import java.io.File;
@@ -56,10 +56,10 @@ public static PackagedProgram createPackagedProgram() throws FlinkException {
}
public static void main(String[] args) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.fromCollection(Arrays.asList(1, 2, 3))
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.fromData(Arrays.asList(1, 2, 3))
.map(element -> element + 1)
- .output(new DiscardingOutputFormat<>());
+ .sinkTo(new DiscardingSink<>());
try {
env.execute();
diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/MultiExecuteJob.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/MultiExecuteJob.java
index 42296187b85b3..0b65361bc0570 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/testjar/MultiExecuteJob.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/MultiExecuteJob.java
@@ -18,11 +18,11 @@
package org.apache.flink.client.testjar;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.io.DiscardingOutputFormat;
import org.apache.flink.client.cli.CliFrontendTestUtils;
import org.apache.flink.client.program.PackagedProgram;
import org.apache.flink.client.program.ProgramInvocationException;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
import org.apache.flink.util.FlinkException;
import java.io.File;
@@ -33,7 +33,8 @@
import java.util.List;
/**
- * A testing job with configurable number of calls to {@link ExecutionEnvironment#executeAsync()}.
+ * A testing job with configurable number of calls to {@link
+ * StreamExecutionEnvironment#executeAsync()}.
*/
public class MultiExecuteJob {
@@ -57,7 +58,7 @@ public static void main(String[] args) throws Exception {
int noOfExecutes = Integer.parseInt(args[0]);
boolean attached = args.length > 1 && Boolean.parseBoolean(args[1]);
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
for (int i = 0; i < noOfExecutes; i++) {
final List input = new ArrayList<>();
@@ -65,9 +66,7 @@ public static void main(String[] args) throws Exception {
input.add(2);
input.add(3);
- env.fromCollection(input)
- .map(element -> element + 1)
- .output(new DiscardingOutputFormat<>());
+ env.fromData(input).map(element -> element + 1).sinkTo(new DiscardingSink<>());
if (attached) {
env.execute();
diff --git a/flink-clients/src/test/java/org/apache/flink/client/testjar/TestJob.java b/flink-clients/src/test/java/org/apache/flink/client/testjar/TestJob.java
index 898d298429cb9..4d9eb8e0f0e69 100644
--- a/flink-clients/src/test/java/org/apache/flink/client/testjar/TestJob.java
+++ b/flink-clients/src/test/java/org/apache/flink/client/testjar/TestJob.java
@@ -18,12 +18,12 @@
package org.apache.flink.client.testjar;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.client.deployment.application.JarManifestParserTest;
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
+import org.apache.flink.util.ParameterTool;
import java.io.File;
import java.io.FileNotFoundException;
diff --git a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
index 1e53eb53cfb6d..93954e1e86431 100644
--- a/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
+++ b/flink-connectors/flink-connector-files/src/main/java/org/apache/flink/connector/file/table/FileSystemTableSource.java
@@ -20,7 +20,6 @@
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.serialization.DeserializationSchema;
-import org.apache.flink.api.java.io.CollectionInputFormat;
import org.apache.flink.configuration.ReadableConfig;
import org.apache.flink.connector.file.src.FileSource;
import org.apache.flink.connector.file.src.FileSourceSplit;
@@ -30,6 +29,7 @@
import org.apache.flink.connector.file.src.reader.BulkFormat;
import org.apache.flink.connector.file.table.format.BulkDecodingFormat;
import org.apache.flink.core.fs.Path;
+import org.apache.flink.streaming.api.legacy.io.CollectionInputFormat;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.api.ValidationException;
diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java
index 0440977d13951..fe098d7660927 100644
--- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/FileSystemOutputFormatTest.java
@@ -19,9 +19,9 @@
package org.apache.flink.connector.file.table;
import org.apache.flink.api.common.io.FinalizeOnMaster.FinalizationContext;
-import org.apache.flink.api.java.io.TextOutputFormat;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.legacy.io.TextOutputFormat;
import org.apache.flink.streaming.api.operators.StreamSink;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
diff --git a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchFileWriterTest.java b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchFileWriterTest.java
index cf1358c4a558f..cedb95538204f 100644
--- a/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchFileWriterTest.java
+++ b/flink-connectors/flink-connector-files/src/test/java/org/apache/flink/connector/file/table/batch/compact/BatchFileWriterTest.java
@@ -18,7 +18,6 @@
package org.apache.flink.connector.file.table.batch.compact;
-import org.apache.flink.api.java.io.TextOutputFormat;
import org.apache.flink.connector.file.table.FileSystemFactory;
import org.apache.flink.connector.file.table.PartitionTempFileManager;
import org.apache.flink.connector.file.table.RowPartitionComputer;
@@ -28,6 +27,7 @@
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.core.fs.Path;
import org.apache.flink.streaming.api.functions.sink.filesystem.OutputFileConfig;
+import org.apache.flink.streaming.api.legacy.io.TextOutputFormat;
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
import org.apache.flink.types.Row;
diff --git a/flink-connectors/flink-connector-hive/pom.xml b/flink-connectors/flink-connector-hive/pom.xml
index e7a47b75477b0..897191ce4393e 100644
--- a/flink-connectors/flink-connector-hive/pom.xml
+++ b/flink-connectors/flink-connector-hive/pom.xml
@@ -703,12 +703,6 @@ under the License.
-
- org.apache.flink
- flink-java
- ${project.version}
- test
- org.apache.flinkflink-clients
diff --git a/flink-connectors/flink-hadoop-compatibility/pom.xml b/flink-connectors/flink-hadoop-compatibility/pom.xml
index 3125760168651..09a0d314d3483 100644
--- a/flink-connectors/flink-hadoop-compatibility/pom.xml
+++ b/flink-connectors/flink-hadoop-compatibility/pom.xml
@@ -39,19 +39,40 @@ under the License.
org.apache.flink
- flink-java
+ flink-core
+ ${project.version}
+ provided
+
+
+
+ org.apache.flink
+ flink-runtime${project.version}providedorg.apache.flink
- flink-java
+ flink-streaming-java
+ ${project.version}
+ provided
+
+
+
+ org.apache.flink
+ flink-streaming-java${project.version}test-jartest
+
+ org.apache.flink
+ flink-connector-files
+ ${project.version}
+ test
+
+
org.apache.hadoophadoop-common
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
index f14b2875e5b96..f5dfc8f21baf8 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/HadoopUtils.java
@@ -18,7 +18,7 @@
package org.apache.flink.hadoopcompatibility;
-import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.util.ParameterTool;
import org.apache.commons.cli.Option;
import org.apache.hadoop.util.GenericOptionsParser;
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
deleted file mode 100644
index fc9981379f8c2..0000000000000
--- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceCombineFunction.java
+++ /dev/null
@@ -1,188 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.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;
-import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
-import org.apache.flink.api.java.tuple.Tuple2;
-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.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
-import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.InstantiationUtil;
-
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.Serializable;
-
-/**
- * This wrapper maps a Hadoop Reducer and Combiner (mapred API) to a combinable Flink
- * GroupReduceFunction.
- */
-@SuppressWarnings("rawtypes")
-@Public
-public final class HadoopReduceCombineFunction
- extends RichGroupReduceFunction, Tuple2>
- implements GroupCombineFunction, Tuple2>,
- ResultTypeQueryable>,
- Serializable {
-
- private static final long serialVersionUID = 1L;
-
- private transient Reducer reducer;
- private transient Reducer combiner;
- private transient JobConf jobConf;
-
- private transient HadoopTupleUnwrappingIterator valueIterator;
- private transient HadoopOutputCollector reduceCollector;
- private transient HadoopOutputCollector combineCollector;
- private transient Reporter reporter;
-
- /**
- * Maps two Hadoop Reducer (mapred API) to a combinable Flink GroupReduceFunction.
- *
- * @param hadoopReducer The Hadoop Reducer that is mapped to a GroupReduceFunction.
- * @param hadoopCombiner The Hadoop Reducer that is mapped to the combiner function.
- */
- public HadoopReduceCombineFunction(
- Reducer hadoopReducer,
- Reducer hadoopCombiner) {
- this(hadoopReducer, hadoopCombiner, new JobConf());
- }
-
- /**
- * Maps two Hadoop Reducer (mapred API) to a combinable Flink GroupReduceFunction.
- *
- * @param hadoopReducer The Hadoop Reducer that is mapped to a GroupReduceFunction.
- * @param hadoopCombiner The Hadoop Reducer that is mapped to the combiner function.
- * @param conf The JobConf that is used to configure both Hadoop Reducers.
- */
- public HadoopReduceCombineFunction(
- Reducer hadoopReducer,
- Reducer hadoopCombiner,
- JobConf conf) {
- if (hadoopReducer == null) {
- throw new NullPointerException("Reducer may not be null.");
- }
- if (hadoopCombiner == null) {
- throw new NullPointerException("Combiner may not be null.");
- }
- if (conf == null) {
- throw new NullPointerException("JobConf may not be null.");
- }
-
- this.reducer = hadoopReducer;
- this.combiner = hadoopCombiner;
- this.jobConf = conf;
- }
-
- @SuppressWarnings("unchecked")
- @PublicEvolving
- @Override
- public void open(OpenContext openContext) throws Exception {
- super.open(openContext);
- this.reducer.configure(jobConf);
- this.combiner.configure(jobConf);
-
- this.reporter = new HadoopDummyReporter();
- Class inKeyClass =
- (Class) TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 0);
- TypeSerializer keySerializer =
- getRuntimeContext().createSerializer(TypeExtractor.getForClass(inKeyClass));
- this.valueIterator = new HadoopTupleUnwrappingIterator<>(keySerializer);
- this.combineCollector = new HadoopOutputCollector<>();
- this.reduceCollector = new HadoopOutputCollector<>();
- }
-
- @Override
- public void reduce(
- final Iterable> values,
- final Collector> out)
- throws Exception {
- reduceCollector.setFlinkCollector(out);
- valueIterator.set(values.iterator());
- reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
- }
-
- @Override
- public void combine(
- final Iterable> values,
- final Collector> out)
- throws Exception {
- combineCollector.setFlinkCollector(out);
- valueIterator.set(values.iterator());
- combiner.reduce(valueIterator.getCurrentKey(), valueIterator, combineCollector, reporter);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public TypeInformation> getProducedType() {
- Class outKeyClass =
- (Class)
- TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 2);
- Class outValClass =
- (Class)
- TypeExtractor.getParameterType(Reducer.class, reducer.getClass(), 3);
-
- final TypeInformation keyTypeInfo = TypeExtractor.getForClass(outKeyClass);
- final TypeInformation valueTypleInfo = TypeExtractor.getForClass(outValClass);
- return new TupleTypeInfo<>(keyTypeInfo, valueTypleInfo);
- }
-
- /**
- * Custom serialization methods.
- *
- * @see http://docs.oracle.com/javase/7/docs/api/java/io/Serializable.html
- */
- private void writeObject(final ObjectOutputStream out) throws IOException {
-
- out.writeObject(reducer.getClass());
- out.writeObject(combiner.getClass());
- jobConf.write(out);
- }
-
- @SuppressWarnings("unchecked")
- private void readObject(final ObjectInputStream in) throws IOException, ClassNotFoundException {
-
- Class> reducerClass =
- (Class>) in.readObject();
- reducer = InstantiationUtil.instantiate(reducerClass);
-
- Class> combinerClass =
- (Class>) in.readObject();
- combiner = InstantiationUtil.instantiate(combinerClass);
-
- jobConf = new JobConf();
- jobConf.readFields(in);
- }
-}
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/HadoopReducerWrappedFunction.java
similarity index 77%
rename from flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReduceFunction.java
rename to flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/HadoopReducerWrappedFunction.java
index 8c3424e2a2690..69032c3e4efe8 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/HadoopReducerWrappedFunction.java
@@ -21,7 +21,6 @@
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;
import org.apache.flink.api.java.hadoop.mapred.wrapper.HadoopDummyReporter;
@@ -31,6 +30,9 @@
import org.apache.flink.api.java.typeutils.TypeExtractor;
import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopOutputCollector;
import org.apache.flink.hadoopcompatibility.mapred.wrapper.HadoopTupleUnwrappingIterator;
+import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction;
+import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction;
+import org.apache.flink.streaming.api.windowing.windows.GlobalWindow;
import org.apache.flink.util.Collector;
import org.apache.flink.util.InstantiationUtil;
@@ -44,13 +46,19 @@
import java.io.Serializable;
/**
- * This wrapper maps a Hadoop Reducer (mapred API) to a non-combinable Flink GroupReduceFunction.
+ * This wrapper maps a Hadoop Reducer (mapred API) to a Flink window function.
+ *
+ *
This function can be utilized in both keyedStream and non-keyed stream.
*/
@SuppressWarnings("rawtypes")
@Public
-public final class HadoopReduceFunction
- extends RichGroupReduceFunction, Tuple2>
- implements ResultTypeQueryable>, Serializable {
+public final class HadoopReducerWrappedFunction
+ extends RichWindowFunction<
+ Tuple2, Tuple2, KEYIN, GlobalWindow>
+ implements AllWindowFunction<
+ Tuple2, Tuple2, GlobalWindow>,
+ ResultTypeQueryable>,
+ Serializable {
private static final long serialVersionUID = 1L;
@@ -66,7 +74,7 @@ public final class HadoopReduceFunction
*
* @param hadoopReducer The Hadoop Reducer to wrap.
*/
- public HadoopReduceFunction(Reducer hadoopReducer) {
+ public HadoopReducerWrappedFunction(Reducer hadoopReducer) {
this(hadoopReducer, new JobConf());
}
@@ -76,7 +84,7 @@ public HadoopReduceFunction(Reducer hadoopRedu
* @param hadoopReducer The Hadoop Reducer to wrap.
* @param conf The JobConf that is used to configure the Hadoop Reducer.
*/
- public HadoopReduceFunction(
+ public HadoopReducerWrappedFunction(
Reducer hadoopReducer, JobConf conf) {
if (hadoopReducer == null) {
throw new NullPointerException("Reducer may not be null.");
@@ -105,17 +113,6 @@ public void open(OpenContext openContext) throws Exception {
this.valueIterator = new HadoopTupleUnwrappingIterator(keySerializer);
}
- @Override
- public void reduce(
- final Iterable> values,
- final Collector> out)
- throws Exception {
-
- reduceCollector.setFlinkCollector(out);
- valueIterator.set(values.iterator());
- reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
- }
-
@SuppressWarnings("unchecked")
@Override
public TypeInformation> getProducedType() {
@@ -155,4 +152,27 @@ private void readObject(final ObjectInputStream in) throws IOException, ClassNot
jobConf = new JobConf();
jobConf.readFields(in);
}
+
+ @Override
+ public void apply(
+ KEYIN text,
+ GlobalWindow globalWindow,
+ Iterable> iterable,
+ Collector> collector)
+ throws Exception {
+ reduceCollector.setFlinkCollector(collector);
+ valueIterator.set(iterable.iterator());
+ reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
+ }
+
+ @Override
+ public void apply(
+ GlobalWindow globalWindow,
+ Iterable> iterable,
+ Collector> collector)
+ throws Exception {
+ reduceCollector.setFlinkCollector(collector);
+ valueIterator.set(iterable.iterator());
+ reducer.reduce(valueIterator.getCurrentKey(), valueIterator, reduceCollector, reporter);
+ }
}
diff --git a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
index e414403a2ce37..5b5cc20dcbd52 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/main/java/org/apache/flink/hadoopcompatibility/mapred/wrapper/HadoopTupleUnwrappingIterator.java
@@ -18,8 +18,8 @@
package org.apache.flink.hadoopcompatibility.mapred.wrapper;
+import org.apache.flink.api.common.operators.util.TupleUnwrappingIterator;
import org.apache.flink.api.common.typeutils.TypeSerializer;
-import org.apache.flink.api.java.operators.translation.TupleUnwrappingIterator;
import org.apache.flink.api.java.tuple.Tuple2;
import java.util.Iterator;
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
index eb416c38b7cd3..029240acafb70 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/hadoopcompatibility/HadoopUtilsTest.java
@@ -18,8 +18,8 @@
package org.apache.flink.hadoopcompatibility;
-import org.apache.flink.api.java.utils.AbstractParameterToolTest;
-import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.util.AbstractParameterToolTest;
+import org.apache.flink.util.ParameterTool;
import org.junit.jupiter.api.Test;
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopIOFormatsITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopIOFormatsITCase.java
index c7f15cc2a2bbe..5996739e93c40 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopIOFormatsITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopIOFormatsITCase.java
@@ -18,11 +18,16 @@
package org.apache.flink.test.hadoopcompatibility.mapred;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.functions.ReduceFunction;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.test.util.JavaProgramTestBase;
import org.apache.flink.testutils.junit.extensions.parameterized.Parameter;
import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
@@ -82,12 +87,6 @@ public void testJobWithoutObjectReuse() throws Exception {
super.testJobWithoutObjectReuse();
}
- @Override
- @TestTemplate
- public void testJobCollectionExecution() throws Exception {
- super.testJobCollectionExecution();
- }
-
@Override
protected void preSubmit() throws Exception {
resultPath = new String[] {getTempDirPath("result0"), getTempDirPath("result1")};
@@ -147,10 +146,12 @@ protected void preSubmit() throws Exception {
}
@Override
- protected void testProgram() throws Exception {
- expectedResult =
+ protected JobExecutionResult testProgram() throws Exception {
+ Tuple2 expectedResultAndJobExecutionResult =
HadoopIOFormatPrograms.runProgram(
curProgId, resultPath, sequenceFileInPath, sequenceFileInPathNull);
+ expectedResult = expectedResultAndJobExecutionResult.f0;
+ return expectedResultAndJobExecutionResult.f1;
}
@Override
@@ -174,7 +175,7 @@ public static Collection getConfigurations() {
private static class HadoopIOFormatPrograms {
- public static String[] runProgram(
+ public static Tuple2 runProgram(
int progId,
String[] resultPath,
String sequenceFileInPath,
@@ -185,8 +186,8 @@ public static String[] runProgram(
case 1:
{
/** Test sequence file, including a key access. */
- final ExecutionEnvironment env =
- ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
SequenceFileInputFormat sfif =
new SequenceFileInputFormat();
@@ -195,8 +196,22 @@ public static String[] runProgram(
HadoopInputFormat hif =
new HadoopInputFormat(
sfif, LongWritable.class, Text.class, hdconf);
- DataSet> ds = env.createInput(hif);
- DataSet> sumed =
+ DataStream> ds = env.createInput(hif);
+
+ ds.map(
+ new MapFunction<
+ Tuple2, Tuple2>() {
+ @Override
+ public Tuple2 map(
+ Tuple2 value)
+ throws Exception {
+ return new Tuple2(
+ value.f0.get(), value.f1);
+ }
+ })
+ .print();
+
+ DataStream> sumed =
ds.map(
new MapFunction<
Tuple2,
@@ -209,10 +224,37 @@ public Tuple2 map(
value.f0.get(), value.f1);
}
})
- .sum(0);
- sumed.writeAsText(resultPath[0]);
- DataSet res =
- ds.distinct(0)
+ .windowAll(GlobalWindows.createWithEndOfStreamTrigger())
+ .reduce(
+ new ReduceFunction>() {
+
+ @Override
+ public Tuple2 reduce(
+ Tuple2 value1,
+ Tuple2 value2) {
+ return Tuple2.of(
+ value1.f0 + value2.f0, value2.f1);
+ }
+ });
+ sumed.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath[0]),
+ new SimpleStringEncoder>())
+ .build());
+
+ DataStream res =
+ ds.keyBy(x -> x.f0)
+ .window(GlobalWindows.createWithEndOfStreamTrigger())
+ .reduce(
+ new ReduceFunction>() {
+ @Override
+ public Tuple2 reduce(
+ Tuple2 value1,
+ Tuple2 value2)
+ throws Exception {
+ return value1;
+ }
+ })
.map(
new MapFunction<
Tuple2, String>() {
@@ -223,22 +265,28 @@ public String map(
return value.f1 + " - " + value.f0.get();
}
});
- res.writeAsText(resultPath[1]);
- env.execute();
+ res.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath[1]),
+ new SimpleStringEncoder())
+ .build());
+ JobExecutionResult jobExecutionResult = env.execute();
// return expected result
- return new String[] {
- "(21,3 - somestring)",
- "0 - somestring - 0\n"
- + "1 - somestring - 1\n"
- + "2 - somestring - 2\n"
- + "3 - somestring - 3\n"
- };
+ return Tuple2.of(
+ new String[] {
+ "(21,3 - somestring)",
+ "0 - somestring - 0\n"
+ + "1 - somestring - 1\n"
+ + "2 - somestring - 2\n"
+ + "3 - somestring - 3\n"
+ },
+ jobExecutionResult);
}
case 2:
{
- final ExecutionEnvironment env =
- ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
SequenceFileInputFormat sfif =
new SequenceFileInputFormat();
@@ -248,8 +296,8 @@ public String map(
HadoopInputFormat hif =
new HadoopInputFormat(
sfif, NullWritable.class, LongWritable.class, hdconf);
- DataSet> ds = env.createInput(hif);
- DataSet> res =
+ DataStream> ds = env.createInput(hif);
+ DataStream> res =
ds.map(
new MapFunction<
Tuple2,
@@ -261,16 +309,28 @@ public Tuple2 map(
return new Tuple2(null, value.f1.get());
}
});
- DataSet> res1 = res.groupBy(1).sum(1);
- res1.writeAsText(resultPath[1]);
- res.writeAsText(resultPath[0]);
- env.execute();
+ DataStream> res1 = res.keyBy(x -> x.f1).sum(1);
+
+ res1.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath[1]),
+ new SimpleStringEncoder>())
+ .build());
+
+ res.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath[0]),
+ new SimpleStringEncoder>())
+ .build());
+ JobExecutionResult jobExecutionResult = env.execute();
// return expected result
- return new String[] {
- "(null,2)\n" + "(null,0)\n" + "(null,1)\n" + "(null,3)",
- "(null,0)\n" + "(null,1)\n" + "(null,2)\n" + "(null,3)"
- };
+ return Tuple2.of(
+ new String[] {
+ "(null,2)\n" + "(null,0)\n" + "(null,1)\n" + "(null,3)",
+ "(null,0)\n" + "(null,1)\n" + "(null,2)\n" + "(null,3)"
+ },
+ jobExecutionResult);
}
default:
throw new IllegalArgumentException("Invalid program id");
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
index 9875b49e33b78..2f9d68e6531c2 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapFunctionITCase.java
@@ -18,11 +18,12 @@
package org.apache.flink.test.hadoopcompatibility.mapred;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.hadoopcompatibility.mapred.HadoopMapFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.test.util.MultipleProgramsTestBase;
import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
@@ -47,17 +48,21 @@ class HadoopMapFunctionITCase extends MultipleProgramsTestBase {
@TestTemplate
void testNonPassingMapper(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataSet> ds = HadoopTestData.getKVPairDataSet(env);
- DataSet> nonPassingFlatMapDs =
+ DataStream> ds = HadoopTestData.getKVPairDataStream(env);
+ DataStream> nonPassingFlatMapDs =
ds.flatMap(
new HadoopMapFunction(
new NonPassingMapper()));
String resultPath = tempFolder.toUri().toString();
- nonPassingFlatMapDs.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
+ nonPassingFlatMapDs.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath),
+ new SimpleStringEncoder>())
+ .build());
env.execute();
compareResultsByLinesInMemory("\n", resultPath);
@@ -65,17 +70,21 @@ void testNonPassingMapper(@TempDir Path tempFolder) throws Exception {
@TestTemplate
void testDataDuplicatingMapper(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataSet> ds = HadoopTestData.getKVPairDataSet(env);
- DataSet> duplicatingFlatMapDs =
+ DataStream> ds = HadoopTestData.getKVPairDataStream(env);
+ DataStream> duplicatingFlatMapDs =
ds.flatMap(
new HadoopMapFunction(
new DuplicatingMapper()));
String resultPath = tempFolder.toUri().toString();
- duplicatingFlatMapDs.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
+ duplicatingFlatMapDs.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath),
+ new SimpleStringEncoder>())
+ .build());
env.execute();
String expected =
@@ -127,20 +136,24 @@ void testDataDuplicatingMapper(@TempDir Path tempFolder) throws Exception {
@TestTemplate
void testConfigurableMapper(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
JobConf conf = new JobConf();
conf.set("my.filterPrefix", "Hello");
- DataSet> ds = HadoopTestData.getKVPairDataSet(env);
- DataSet> hellos =
+ DataStream> ds = HadoopTestData.getKVPairDataStream(env);
+ DataStream> hellos =
ds.flatMap(
new HadoopMapFunction(
new ConfigurableMapper(), conf));
String resultPath = tempFolder.toUri().toString();
- hellos.writeAsText(resultPath, FileSystem.WriteMode.OVERWRITE);
+ hellos.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath),
+ new SimpleStringEncoder>())
+ .build());
env.execute();
String expected = "(2,Hello)\n" + "(3,Hello world)\n" + "(4,Hello world, how are you?)\n";
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
index 0f7e284ba1e19..11809b5b4b90e 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopMapredITCase.java
@@ -18,6 +18,7 @@
package org.apache.flink.test.hadoopcompatibility.mapred;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.test.hadoopcompatibility.mapred.example.HadoopMapredCompatWordCount;
import org.apache.flink.test.testdata.WordCountData;
import org.apache.flink.test.util.JavaProgramTestBase;
@@ -57,7 +58,7 @@ protected void postSubmit() throws Exception {
}
@Override
- protected void testProgram() throws Exception {
- HadoopMapredCompatWordCount.main(new String[] {textPath, resultPath});
+ protected JobExecutionResult testProgram() throws Exception {
+ return HadoopMapredCompatWordCount.run(new String[] {textPath, resultPath});
}
}
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
deleted file mode 100644
index 3d7376c61722c..0000000000000
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceCombineFunctionITCase.java
+++ /dev/null
@@ -1,284 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.test.hadoopcompatibility.mapred;
-
-import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceCombineFunction;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceFunction;
-import org.apache.flink.test.util.MultipleProgramsTestBase;
-import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
-
-import org.apache.hadoop.io.IntWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.OutputCollector;
-import org.apache.hadoop.mapred.Reducer;
-import org.apache.hadoop.mapred.Reporter;
-import org.junit.jupiter.api.TestTemplate;
-import org.junit.jupiter.api.extension.ExtendWith;
-import org.junit.jupiter.api.io.TempDir;
-
-import java.io.IOException;
-import java.nio.file.Path;
-import java.util.Iterator;
-
-import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory;
-import static org.assertj.core.api.Assumptions.assumeThat;
-
-/** IT case for the {@link HadoopReduceCombineFunction}. */
-@ExtendWith(ParameterizedTestExtension.class)
-class HadoopReduceCombineFunctionITCase extends MultipleProgramsTestBase {
-
- @TestTemplate
- void testStandardCountingWithCombiner(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- DataSet> ds =
- HadoopTestData.getKVPairDataSet(env).map(new Mapper1());
-
- DataSet> counts =
- ds.groupBy(0)
- .reduceGroup(
- new HadoopReduceCombineFunction<
- IntWritable, IntWritable, IntWritable, IntWritable>(
- new SumReducer(), new SumReducer()));
-
- String resultPath = tempFolder.toUri().toString();
-
- counts.writeAsText(resultPath);
- env.execute();
-
- String expected = "(0,5)\n" + "(1,6)\n" + "(2,6)\n" + "(3,4)\n";
-
- compareResultsByLinesInMemory(expected, resultPath);
- }
-
- @TestTemplate
- void testUngroupedHadoopReducer(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- DataSet> ds =
- HadoopTestData.getKVPairDataSet(env).map(new Mapper2());
-
- DataSet> sum =
- ds.reduceGroup(
- new HadoopReduceCombineFunction<
- IntWritable, IntWritable, IntWritable, IntWritable>(
- new SumReducer(), new SumReducer()));
-
- String resultPath = tempFolder.toUri().toString();
-
- sum.writeAsText(resultPath);
- env.execute();
-
- String expected = "(0,231)\n";
-
- compareResultsByLinesInMemory(expected, resultPath);
- }
-
- @TestTemplate
- void testCombiner(@TempDir Path tempFolder) throws Exception {
- assumeThat(mode).isEqualTo(TestExecutionMode.CLUSTER);
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- DataSet> ds =
- HadoopTestData.getKVPairDataSet(env).map(new Mapper3());
-
- DataSet> counts =
- ds.groupBy(0)
- .reduceGroup(
- new HadoopReduceCombineFunction<
- IntWritable, IntWritable, IntWritable, IntWritable>(
- new SumReducer(), new KeyChangingReducer()));
-
- String resultPath = tempFolder.toUri().toString();
-
- counts.writeAsText(resultPath);
- env.execute();
-
- String expected = "(0,5)\n" + "(1,6)\n" + "(2,5)\n" + "(3,5)\n";
-
- compareResultsByLinesInMemory(expected, resultPath);
- }
-
- @TestTemplate
- void testConfigurationViaJobConf(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- JobConf conf = new JobConf();
- conf.set("my.cntPrefix", "Hello");
-
- DataSet> ds =
- HadoopTestData.getKVPairDataSet(env).map(new Mapper4());
-
- DataSet> hellos =
- ds.groupBy(0)
- .reduceGroup(
- new HadoopReduceFunction<
- IntWritable, Text, IntWritable, IntWritable>(
- new ConfigurableCntReducer(), conf));
-
- String resultPath = tempFolder.toUri().toString();
-
- hellos.writeAsText(resultPath);
- env.execute();
-
- // return expected result
- String expected = "(0,0)\n" + "(1,0)\n" + "(2,1)\n" + "(3,1)\n" + "(4,1)\n";
-
- compareResultsByLinesInMemory(expected, resultPath);
- }
-
- /** A {@link Reducer} to sum counts. */
- public static class SumReducer
- implements Reducer {
-
- @Override
- public void reduce(
- IntWritable k,
- Iterator v,
- OutputCollector out,
- Reporter r)
- throws IOException {
-
- int sum = 0;
- while (v.hasNext()) {
- sum += v.next().get();
- }
- out.collect(k, new IntWritable(sum));
- }
-
- @Override
- public void configure(JobConf arg0) {}
-
- @Override
- public void close() throws IOException {}
- }
-
- /** A {@link Reducer} to sum counts that modifies the key. */
- public static class KeyChangingReducer
- implements Reducer {
-
- @Override
- public void reduce(
- IntWritable k,
- Iterator v,
- OutputCollector out,
- Reporter r)
- throws IOException {
- while (v.hasNext()) {
- out.collect(new IntWritable(k.get() % 4), v.next());
- }
- }
-
- @Override
- public void configure(JobConf arg0) {}
-
- @Override
- public void close() throws IOException {}
- }
-
- /** A {@link Reducer} to sum counts for a specific prefix. */
- public static class ConfigurableCntReducer
- implements Reducer {
- private String countPrefix;
-
- @Override
- public void reduce(
- IntWritable k,
- Iterator vs,
- OutputCollector out,
- Reporter r)
- throws IOException {
- int commentCnt = 0;
- while (vs.hasNext()) {
- String v = vs.next().toString();
- if (v.startsWith(this.countPrefix)) {
- commentCnt++;
- }
- }
- out.collect(k, new IntWritable(commentCnt));
- }
-
- @Override
- public void configure(final JobConf c) {
- this.countPrefix = c.get("my.cntPrefix");
- }
-
- @Override
- public void close() throws IOException {}
- }
-
- /** Test mapper. */
- public static class Mapper1
- implements MapFunction, Tuple2> {
- private static final long serialVersionUID = 1L;
- Tuple2 outT = new Tuple2();
-
- @Override
- public Tuple2 map(Tuple2 v) throws Exception {
- outT.f0 = new IntWritable(v.f0.get() / 6);
- outT.f1 = new IntWritable(1);
- return outT;
- }
- }
-
- /** Test mapper. */
- public static class Mapper2
- implements MapFunction, Tuple2> {
- private static final long serialVersionUID = 1L;
- Tuple2 outT = new Tuple2();
-
- @Override
- public Tuple2 map(Tuple2 v) throws Exception {
- outT.f0 = new IntWritable(0);
- outT.f1 = v.f0;
- return outT;
- }
- }
-
- /** Test mapper. */
- public static class Mapper3
- implements MapFunction, Tuple2> {
- private static final long serialVersionUID = 1L;
- Tuple2 outT = new Tuple2();
-
- @Override
- public Tuple2 map(Tuple2 v) throws Exception {
- outT.f0 = v.f0;
- outT.f1 = new IntWritable(1);
- return outT;
- }
- }
-
- /** Test mapper. */
- public static class Mapper4
- implements MapFunction, Tuple2> {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Tuple2 map(Tuple2 v) throws Exception {
- v.f0 = new IntWritable(v.f0.get() % 5);
- return v;
- }
- }
-}
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
index 4e991f16118f6..47445ed31be59 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopReduceFunctionITCase.java
@@ -18,11 +18,16 @@
package org.apache.flink.test.hadoopcompatibility.mapred;
+import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceFunction;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopReducerWrappedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.test.util.MultipleProgramsTestBase;
import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
@@ -42,27 +47,30 @@
import static org.apache.flink.test.util.TestBaseUtils.compareResultsByLinesInMemory;
-/** IT cases for the {@link HadoopReduceFunction}. */
+/** IT cases for the {@link HadoopReducerWrappedFunction}. */
@ExtendWith(ParameterizedTestExtension.class)
class HadoopReduceFunctionITCase extends MultipleProgramsTestBase {
@TestTemplate
void testStandardGrouping(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setRuntimeMode(RuntimeExecutionMode.BATCH);
- DataSet> ds =
- HadoopTestData.getKVPairDataSet(env).map(new Mapper1());
+ DataStream> ds =
+ HadoopTestData.getKVPairDataStream(env).map(new Mapper1());
- DataSet> commentCnts =
- ds.groupBy(0)
- .reduceGroup(
- new HadoopReduceFunction<
- IntWritable, Text, IntWritable, IntWritable>(
- new CommentCntReducer()));
+ DataStream> commentCnts =
+ ds.keyBy(x -> x.f0)
+ .window(GlobalWindows.createWithEndOfStreamTrigger())
+ .apply(new HadoopReducerWrappedFunction<>(new CommentCntReducer()));
String resultPath = tempFolder.toUri().toString();
- commentCnts.writeAsText(resultPath);
+ commentCnts.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath),
+ new SimpleStringEncoder>())
+ .build());
env.execute();
String expected = "(0,0)\n" + "(1,3)\n" + "(2,5)\n" + "(3,5)\n" + "(4,2)\n";
@@ -72,18 +80,22 @@ void testStandardGrouping(@TempDir Path tempFolder) throws Exception {
@TestTemplate
void testUngroupedHadoopReducer(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setRuntimeMode(RuntimeExecutionMode.BATCH);
- DataSet> ds = HadoopTestData.getKVPairDataSet(env);
+ DataStream> ds = HadoopTestData.getKVPairDataStream(env);
- DataSet> commentCnts =
- ds.reduceGroup(
- new HadoopReduceFunction(
- new AllCommentCntReducer()));
+ SingleOutputStreamOperator> commentCnts =
+ ds.windowAll(GlobalWindows.createWithEndOfStreamTrigger())
+ .apply(new HadoopReducerWrappedFunction<>(new AllCommentCntReducer()));
String resultPath = tempFolder.toUri().toString();
- commentCnts.writeAsText(resultPath);
+ commentCnts.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath),
+ new SimpleStringEncoder>())
+ .build());
env.execute();
String expected = "(42,15)\n";
@@ -93,24 +105,29 @@ void testUngroupedHadoopReducer(@TempDir Path tempFolder) throws Exception {
@TestTemplate
void testConfigurationViaJobConf(@TempDir Path tempFolder) throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.setRuntimeMode(RuntimeExecutionMode.BATCH);
JobConf conf = new JobConf();
conf.set("my.cntPrefix", "Hello");
- DataSet> ds =
- HadoopTestData.getKVPairDataSet(env).map(new Mapper2());
+ DataStream> ds =
+ HadoopTestData.getKVPairDataStream(env).map(new Mapper2());
- DataSet> helloCnts =
- ds.groupBy(0)
- .reduceGroup(
- new HadoopReduceFunction<
- IntWritable, Text, IntWritable, IntWritable>(
+ DataStream> helloCnts =
+ ds.keyBy(x -> x.f0)
+ .window(GlobalWindows.createWithEndOfStreamTrigger())
+ .apply(
+ new HadoopReducerWrappedFunction<>(
new ConfigurableCntReducer(), conf));
String resultPath = tempFolder.toUri().toString();
- helloCnts.writeAsText(resultPath);
+ helloCnts.sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(resultPath),
+ new SimpleStringEncoder>())
+ .build());
env.execute();
String expected = "(0,0)\n" + "(1,0)\n" + "(2,1)\n" + "(3,1)\n" + "(4,1)\n";
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
index 292eef501e0d9..c60f7f86abfb2 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/HadoopTestData.java
@@ -18,9 +18,9 @@
package org.apache.flink.test.hadoopcompatibility.mapred;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.hadoop.io.IntWritable;
import org.apache.hadoop.io.Text;
@@ -32,7 +32,8 @@
/** Test data. */
public class HadoopTestData {
- public static DataSet> getKVPairDataSet(ExecutionEnvironment env) {
+ public static DataStream> getKVPairDataStream(
+ StreamExecutionEnvironment env) {
List> data = new ArrayList>();
data.add(new Tuple2(new IntWritable(1), new Text("Hi")));
@@ -61,6 +62,6 @@ public static DataSet> getKVPairDataSet(ExecutionEnvir
Collections.shuffle(data);
- return env.fromCollection(data);
+ return env.fromData(data);
}
}
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/WordCountMapredITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/WordCountMapredITCase.java
index 9ab1b95ff8b31..3467a5a75562c 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/WordCountMapredITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/WordCountMapredITCase.java
@@ -18,13 +18,16 @@
package org.apache.flink.test.hadoopcompatibility.mapred;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.hadoopcompatibility.HadoopInputs;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.test.testdata.WordCountData;
import org.apache.flink.test.util.JavaProgramTestBase;
import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
@@ -70,22 +73,23 @@ protected void postSubmit() throws Exception {
}
@Override
- protected void testProgram() throws Exception {
- internalRun();
+ protected JobExecutionResult testProgram() throws Exception {
+ JobExecutionResult jobExecutionResult = internalRun();
postSubmit();
+ return jobExecutionResult;
}
- private void internalRun() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ private JobExecutionResult internalRun() throws Exception {
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataSet> input;
+ DataStream> input;
input =
env.createInput(
HadoopInputs.readHadoopFile(
new TextInputFormat(), LongWritable.class, Text.class, textPath));
- DataSet text =
+ DataStream text =
input.map(
new MapFunction, String>() {
@Override
@@ -94,14 +98,15 @@ public String map(Tuple2 value) throws Exception {
}
});
- DataSet> counts =
+ DataStream> counts =
// split up the lines in pairs (2-tuples) containing: (word,1)
text.flatMap(new Tokenizer())
- // group by the tuple field "0" and sum up tuple field "1"
- .groupBy(0)
+ // key by the tuple field "f0" and sum up tuple field "f1"
+ .keyBy(x -> x.f0)
+ .window(GlobalWindows.createWithEndOfStreamTrigger())
.sum(1);
- DataSet> words =
+ DataStream> words =
counts.map(
new MapFunction, Tuple2>() {
@@ -121,8 +126,8 @@ public Tuple2 map(Tuple2 value)
TextOutputFormat.setOutputPath(hadoopOutputFormat.getJobConf(), new Path(resultPath));
// Output & Execute
- words.output(hadoopOutputFormat);
- env.execute("Hadoop Compat WordCount");
+ words.addSink(new OutputFormatSinkFunction<>(hadoopOutputFormat));
+ return env.execute("Hadoop Compat WordCount");
}
static final class Tokenizer implements FlatMapFunction> {
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
index 1c16675f688b2..829147515db0b 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapred/example/HadoopMapredCompatWordCount.java
@@ -18,13 +18,16 @@
package org.apache.flink.test.hadoopcompatibility.mapred.example;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.java.hadoop.mapred.HadoopInputFormat;
import org.apache.flink.api.java.hadoop.mapred.HadoopOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.hadoopcompatibility.mapred.HadoopMapFunction;
-import org.apache.flink.hadoopcompatibility.mapred.HadoopReduceCombineFunction;
+import org.apache.flink.hadoopcompatibility.mapred.HadoopReducerWrappedFunction;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
@@ -49,16 +52,16 @@
*/
public class HadoopMapredCompatWordCount {
- public static void main(String[] args) throws Exception {
+ public static JobExecutionResult run(String[] args) throws Exception {
if (args.length < 2) {
System.err.println("Usage: WordCount ");
- return;
+ return null;
}
final String inputPath = args[0];
final String outputPath = args[1];
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
// Set up the Hadoop Input Format
HadoopInputFormat hadoopInputFormat =
@@ -67,17 +70,15 @@ public static void main(String[] args) throws Exception {
TextInputFormat.addInputPath(hadoopInputFormat.getJobConf(), new Path(inputPath));
// Create a Flink job with it
- DataSet> text = env.createInput(hadoopInputFormat);
+ DataStream> text = env.createInput(hadoopInputFormat);
- DataSet> words =
+ DataStream> words =
text.flatMap(
new HadoopMapFunction(
new Tokenizer()))
- .groupBy(0)
- .reduceGroup(
- new HadoopReduceCombineFunction<
- Text, LongWritable, Text, LongWritable>(
- new Counter(), new Counter()));
+ .keyBy(x -> x.f0)
+ .window(GlobalWindows.createWithEndOfStreamTrigger())
+ .apply(new HadoopReducerWrappedFunction<>(new Counter()));
// Set up Hadoop Output Format
HadoopOutputFormat hadoopOutputFormat =
@@ -87,8 +88,8 @@ public static void main(String[] args) throws Exception {
TextOutputFormat.setOutputPath(hadoopOutputFormat.getJobConf(), new Path(outputPath));
// Output & Execute
- words.output(hadoopOutputFormat).setParallelism(1);
- env.execute("Hadoop Compat WordCount");
+ words.addSink(new OutputFormatSinkFunction<>(hadoopOutputFormat)).setParallelism(1);
+ return env.execute("Hadoop Compat WordCount");
}
/** A {@link Mapper} that splits a line into words. */
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
index 1eb8cc5c568c1..c427af8e31b0b 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/HadoopInputOutputITCase.java
@@ -18,6 +18,7 @@
package org.apache.flink.test.hadoopcompatibility.mapreduce;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat;
import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
import org.apache.flink.test.hadoopcompatibility.mapreduce.example.WordCount;
@@ -56,7 +57,7 @@ protected void postSubmit() throws Exception {
}
@Override
- protected void testProgram() throws Exception {
- WordCount.main(new String[] {textPath, resultPath});
+ protected JobExecutionResult testProgram() throws Exception {
+ return WordCount.run(new String[] {textPath, resultPath});
}
}
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/WordCountMapreduceITCase.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/WordCountMapreduceITCase.java
index 716c4d16ed0c9..d7c44843da1b5 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/WordCountMapreduceITCase.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/WordCountMapreduceITCase.java
@@ -18,13 +18,16 @@
package org.apache.flink.test.hadoopcompatibility.mapreduce;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.hadoopcompatibility.HadoopInputs;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.test.testdata.WordCountData;
import org.apache.flink.test.util.JavaProgramTestBase;
import org.apache.flink.util.Collector;
@@ -67,21 +70,22 @@ protected void postSubmit() throws Exception {
}
@Override
- protected void testProgram() throws Exception {
- internalRun();
+ protected JobExecutionResult testProgram() throws Exception {
+ JobExecutionResult jobExecutionResult = internalRun();
postSubmit();
+ return jobExecutionResult;
}
- private void internalRun() throws Exception {
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ private JobExecutionResult internalRun() throws Exception {
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- DataSet> input;
+ DataStream> input;
input =
env.createInput(
HadoopInputs.readHadoopFile(
new TextInputFormat(), LongWritable.class, Text.class, textPath));
- DataSet text =
+ DataStream text =
input.map(
new MapFunction, String>() {
@Override
@@ -90,14 +94,15 @@ public String map(Tuple2 value) throws Exception {
}
});
- DataSet> counts =
+ DataStream> counts =
// split up the lines in pairs (2-tuples) containing: (word,1)
text.flatMap(new Tokenizer())
- // group by the tuple field "0" and sum up tuple field "1"
- .groupBy(0)
+ // key by the tuple field "f0" and sum up tuple field "f1"
+ .keyBy(x -> x.f0)
+ .window(GlobalWindows.createWithEndOfStreamTrigger())
.sum(1);
- DataSet> words =
+ DataStream> words =
counts.map(
new MapFunction, Tuple2>() {
@@ -118,8 +123,8 @@ public Tuple2 map(Tuple2 value)
TextOutputFormat.setOutputPath(job, new Path(resultPath));
// Output & Execute
- words.output(hadoopOutputFormat);
- env.execute("Hadoop Compat WordCount");
+ words.addSink(new OutputFormatSinkFunction<>(hadoopOutputFormat));
+ return env.execute("Hadoop Compat WordCount");
}
static final class Tokenizer implements FlatMapFunction> {
diff --git a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
index 0d77aa49d128c..536f877465745 100644
--- a/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
+++ b/flink-connectors/flink-hadoop-compatibility/src/test/java/org/apache/flink/test/hadoopcompatibility/mapreduce/example/WordCount.java
@@ -18,14 +18,16 @@
package org.apache.flink.test.hadoopcompatibility.mapreduce.example;
+import org.apache.flink.api.common.JobExecutionResult;
import org.apache.flink.api.common.functions.RichFlatMapFunction;
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.aggregation.Aggregations;
import org.apache.flink.api.java.hadoop.mapreduce.HadoopInputFormat;
import org.apache.flink.api.java.hadoop.mapreduce.HadoopOutputFormat;
import org.apache.flink.api.java.tuple.Tuple2;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.legacy.OutputFormatSinkFunction;
+import org.apache.flink.streaming.api.windowing.assigners.GlobalWindows;
import org.apache.flink.util.Collector;
import org.apache.hadoop.fs.Path;
@@ -46,16 +48,16 @@
@SuppressWarnings("serial")
public class WordCount {
- public static void main(String[] args) throws Exception {
+ public static JobExecutionResult run(String[] args) throws Exception {
if (args.length < 2) {
System.err.println("Usage: WordCount ");
- return;
+ return null;
}
final String inputPath = args[0];
final String outputPath = args[1];
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
+ final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
// Set up the Hadoop Input Format
Job job = Job.getInstance();
@@ -65,16 +67,17 @@ public static void main(String[] args) throws Exception {
TextInputFormat.addInputPath(job, new Path(inputPath));
// Create a Flink job with it
- DataSet> text = env.createInput(hadoopInputFormat);
+ DataStream> text = env.createInput(hadoopInputFormat);
// Tokenize the line and convert from Writable "Text" to String for better handling
- DataSet> words = text.flatMap(new Tokenizer());
+ DataStream> words = text.flatMap(new Tokenizer());
// Sum up the words
- DataSet> result = words.groupBy(0).aggregate(Aggregations.SUM, 1);
+ DataStream> result =
+ words.keyBy(x -> x.f0).window(GlobalWindows.createWithEndOfStreamTrigger()).sum(1);
// Convert String back to Writable "Text" for use with Hadoop Output Format
- DataSet> hadoopResult = result.map(new HadoopDatatypeMapper());
+ DataStream> hadoopResult = result.map(new HadoopDatatypeMapper());
// Set up Hadoop Output Format
HadoopOutputFormat hadoopOutputFormat =
@@ -91,8 +94,8 @@ public static void main(String[] args) throws Exception {
TextOutputFormat.setOutputPath(job, new Path(outputPath));
// Output & Execute
- hadoopResult.output(hadoopOutputFormat);
- env.execute("Word Count");
+ hadoopResult.addSink(new OutputFormatSinkFunction<>(hadoopOutputFormat));
+ return env.execute("Word Count");
}
/** Splits a line into words and converts Hadoop Writables into normal Java data types. */
diff --git a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
index b2adee5ec224f..fc8c3480c2473 100644
--- a/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
@@ -534,31 +534,6 @@ private void setGlobalJobParameters(Map parameters) {
configuration.set(PipelineOptions.GLOBAL_JOB_PARAMETERS, parameters);
}
- // --------------------------------------------------------------------------------------------
- // Registry for types and serializers
- // --------------------------------------------------------------------------------------------
-
- /**
- * Get if the auto type registration is disabled.
- *
- * @return if the auto type registration is disabled.
- * @deprecated The method is deprecated because it's only used in DataSet API. All Flink DataSet
- * APIs are deprecated since Flink 1.18 and will be removed in a future Flink major version.
- * You can still build your application in DataSet, but you should move to either the
- * DataStream and/or Table API.
- * @see
- * FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet
- * API
- */
- @Deprecated
- public boolean isAutoTypeRegistrationDisabled() {
- return !configuration.get(PipelineOptions.AUTO_TYPE_REGISTRATION);
- }
-
- private void setAutoTypeRegistration(Boolean autoTypeRegistration) {
- configuration.set(PipelineOptions.AUTO_TYPE_REGISTRATION, autoTypeRegistration);
- }
-
public boolean isUseSnapshotCompression() {
return configuration.get(ExecutionOptions.SNAPSHOT_COMPRESSION);
}
@@ -704,9 +679,6 @@ public InlineElement getDescription() {
* @param classLoader a class loader to use when loading classes
*/
public void configure(ReadableConfig configuration, ClassLoader classLoader) {
- configuration
- .getOptional(PipelineOptions.AUTO_TYPE_REGISTRATION)
- .ifPresent(this::setAutoTypeRegistration);
configuration
.getOptional(PipelineOptions.AUTO_GENERATE_UIDS)
.ifPresent(this::setAutoGeneratedUids);
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/TupleUnwrappingIterator.java
similarity index 97%
rename from flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java
rename to flink-core/src/main/java/org/apache/flink/api/common/operators/util/TupleUnwrappingIterator.java
index 8549f344c930b..ecea92fba72f1 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/operators/translation/TupleUnwrappingIterator.java
+++ b/flink-core/src/main/java/org/apache/flink/api/common/operators/util/TupleUnwrappingIterator.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.operators.translation;
+package org.apache.flink.api.common.operators.util;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.java.tuple.Tuple2;
diff --git a/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java
index 984505dbafb23..716982b87a5c0 100644
--- a/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java
+++ b/flink-core/src/main/java/org/apache/flink/configuration/PipelineOptions.java
@@ -86,27 +86,6 @@ public class PipelineOptions {
+ " without discarding state.")
.build());
- /**
- * An option to control whether Flink is automatically registering all types in the user
- * programs with Kryo.
- *
- * @deprecated The config is deprecated because it's only used in DataSet API. All Flink DataSet
- * APIs are deprecated since Flink 1.18 and will be removed in a future Flink major version.
- * You can still build your application in DataSet, but you should move to either the
- * DataStream and/or Table API.
- * @see
- * FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet
- * API
- */
- @Deprecated
- public static final ConfigOption AUTO_TYPE_REGISTRATION =
- key("pipeline.auto-type-registration")
- .booleanType()
- .defaultValue(true)
- .withDescription(
- "Controls whether Flink is automatically registering all types in the user programs"
- + " with Kryo.");
-
public static final ConfigOption AUTO_WATERMARK_INTERVAL =
key("pipeline.auto-watermark-interval")
.durationType()
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java b/flink-core/src/main/java/org/apache/flink/util/AbstractParameterTool.java
similarity index 99%
rename from flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java
rename to flink-core/src/main/java/org/apache/flink/util/AbstractParameterTool.java
index 27cd36b58182a..0f0cd3ff2aed4 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/utils/AbstractParameterTool.java
+++ b/flink-core/src/main/java/org/apache/flink/util/AbstractParameterTool.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.utils;
+package org.apache.flink.util;
import org.apache.flink.annotation.Public;
import org.apache.flink.annotation.PublicEvolving;
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java b/flink-core/src/main/java/org/apache/flink/util/MultipleParameterTool.java
similarity index 98%
rename from flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java
rename to flink-core/src/main/java/org/apache/flink/util/MultipleParameterTool.java
index eb0f256032f37..60500646eea4c 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/utils/MultipleParameterTool.java
+++ b/flink-core/src/main/java/org/apache/flink/util/MultipleParameterTool.java
@@ -16,12 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.utils;
+package org.apache.flink.util;
import org.apache.flink.annotation.PublicEvolving;
-import org.apache.flink.api.java.Utils;
-import org.apache.flink.util.CollectionUtil;
-import org.apache.flink.util.Preconditions;
import org.apache.commons.lang3.math.NumberUtils;
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java b/flink-core/src/main/java/org/apache/flink/util/ParameterTool.java
similarity index 97%
rename from flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
rename to flink-core/src/main/java/org/apache/flink/util/ParameterTool.java
index 63064e3b242de..c6f429c5fc934 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/utils/ParameterTool.java
+++ b/flink-core/src/main/java/org/apache/flink/util/ParameterTool.java
@@ -16,13 +16,10 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.utils;
+package org.apache.flink.util;
import org.apache.flink.annotation.Public;
-import org.apache.flink.api.java.Utils;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.CollectionUtil;
-import org.apache.flink.util.Preconditions;
import org.apache.commons.lang3.math.NumberUtils;
@@ -77,14 +74,14 @@ public static ParameterTool fromArgs(String[] args) {
i += 1; // try to find the value
if (i >= args.length) {
- map.put(key, NO_VALUE_KEY);
+ map.put(key, AbstractParameterTool.NO_VALUE_KEY);
} else if (NumberUtils.isNumber(args[i])) {
map.put(key, args[i]);
i += 1;
} else if (args[i].startsWith("--") || args[i].startsWith("-")) {
// the argument cannot be a negative number because we checked earlier
// -> the next argument is a parameter name
- map.put(key, NO_VALUE_KEY);
+ map.put(key, AbstractParameterTool.NO_VALUE_KEY);
} else {
map.put(key, args[i]);
i += 1;
diff --git a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java b/flink-core/src/main/java/org/apache/flink/util/Utils.java
similarity index 99%
rename from flink-java/src/main/java/org/apache/flink/api/java/Utils.java
rename to flink-core/src/main/java/org/apache/flink/util/Utils.java
index 77e952dad0af4..cda91d9401ca6 100644
--- a/flink-java/src/main/java/org/apache/flink/api/java/Utils.java
+++ b/flink-core/src/main/java/org/apache/flink/util/Utils.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.api.java;
+package org.apache.flink.util;
import org.apache.flink.annotation.Internal;
import org.apache.flink.api.common.accumulators.Accumulator;
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java b/flink-core/src/test/java/org/apache/flink/util/AbstractParameterToolTest.java
similarity index 99%
rename from flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java
rename to flink-core/src/test/java/org/apache/flink/util/AbstractParameterToolTest.java
index f45815ad21e8f..bb3f95415b5a6 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/utils/AbstractParameterToolTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/AbstractParameterToolTest.java
@@ -16,12 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.utils;
+package org.apache.flink.util;
import org.apache.flink.api.java.ClosureCleaner;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.util.InstantiationUtil;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.java b/flink-core/src/test/java/org/apache/flink/util/MultipleParameterToolTest.java
similarity index 99%
rename from flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.java
rename to flink-core/src/test/java/org/apache/flink/util/MultipleParameterToolTest.java
index 1161bfbcd7fab..70ca621f80bcb 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/utils/MultipleParameterToolTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/MultipleParameterToolTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.utils;
+package org.apache.flink.util;
import org.junit.jupiter.api.Test;
diff --git a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java b/flink-core/src/test/java/org/apache/flink/util/ParameterToolTest.java
similarity index 99%
rename from flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
rename to flink-core/src/test/java/org/apache/flink/util/ParameterToolTest.java
index 7599219664a88..70a2323b614a8 100644
--- a/flink-java/src/test/java/org/apache/flink/api/java/utils/ParameterToolTest.java
+++ b/flink-core/src/test/java/org/apache/flink/util/ParameterToolTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.api.java.utils;
+package org.apache.flink.util;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
diff --git a/flink-datastream/src/main/java/org/apache/flink/datastream/impl/utils/StreamUtils.java b/flink-datastream/src/main/java/org/apache/flink/datastream/impl/utils/StreamUtils.java
index 8a7ee569c9e3f..688b24e7fb1de 100644
--- a/flink-datastream/src/main/java/org/apache/flink/datastream/impl/utils/StreamUtils.java
+++ b/flink-datastream/src/main/java/org/apache/flink/datastream/impl/utils/StreamUtils.java
@@ -23,7 +23,6 @@
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.connector.dsv2.Sink;
import org.apache.flink.api.connector.dsv2.WrappedSink;
-import org.apache.flink.api.java.Utils;
import org.apache.flink.api.java.functions.KeySelector;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.TypeExtractor;
@@ -49,6 +48,7 @@
import org.apache.flink.streaming.api.transformations.DataStreamV2SinkTransformation;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
+import org.apache.flink.util.Utils;
import java.util.Set;
diff --git a/flink-dist/pom.xml b/flink-dist/pom.xml
index 7059348a7b78e..c72935206a3db 100644
--- a/flink-dist/pom.xml
+++ b/flink-dist/pom.xml
@@ -75,13 +75,6 @@ under the License.
${flink.markBundledAsOptional}
-
- org.apache.flink
- flink-optimizer
- ${project.version}
- ${flink.markBundledAsOptional}
-
-
org.apache.flinkflink-clients
@@ -242,13 +235,6 @@ under the License.
they are not included into the 'flink-dist' uber jar.
-->
-
- org.apache.flink
- flink-examples-batch
- ${project.version}
- provided
-
-
org.apache.flinkflink-examples-streaming
diff --git a/flink-dist/src/main/assemblies/bin.xml b/flink-dist/src/main/assemblies/bin.xml
index 3ac51f4562ffc..f138f241f6014 100644
--- a/flink-dist/src/main/assemblies/bin.xml
+++ b/flink-dist/src/main/assemblies/bin.xml
@@ -230,24 +230,6 @@ under the License.
0644
-
-
- ../flink-examples/flink-examples-batch/target
- examples/batch
- 0644
-
- *.jar
-
-
- flink-examples-batch*.jar
- original-flink-examples-batch*.jar
-
-
-
../flink-examples/flink-examples-streaming/target
diff --git a/flink-end-to-end-tests/flink-cli-test/pom.xml b/flink-end-to-end-tests/flink-cli-test/pom.xml
index 88da765ad173b..55f75f9f32b53 100644
--- a/flink-end-to-end-tests/flink-cli-test/pom.xml
+++ b/flink-end-to-end-tests/flink-cli-test/pom.xml
@@ -40,6 +40,12 @@
${project.version}provided
+
+ org.apache.flink
+ flink-connector-files
+ ${project.version}
+ provided
+
diff --git a/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java b/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java
index e5d212d54112b..eacf7ddeae762 100644
--- a/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java
+++ b/flink-end-to-end-tests/flink-cli-test/src/main/java/org/apache/flink/streaming/tests/PeriodicStreamingJob.java
@@ -18,6 +18,7 @@
package org.apache.flink.streaming.tests;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeinfo.TypeInformation;
@@ -26,8 +27,8 @@
import org.apache.flink.api.java.tuple.Tuple;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.connector.file.sink.FileSink;
+import org.apache.flink.core.fs.Path;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
@@ -35,6 +36,7 @@
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction;
import org.apache.flink.streaming.api.windowing.assigners.TumblingProcessingTimeWindows;
+import org.apache.flink.util.ParameterTool;
import java.time.Duration;
import java.util.Collections;
@@ -71,7 +73,11 @@ public static void main(String[] args) throws Exception {
.window(TumblingProcessingTimeWindows.of(Duration.ofSeconds(5)))
.sum(0);
- result.writeAsText(outputPath + "/result.txt", FileSystem.WriteMode.OVERWRITE)
+ result.sinkTo(
+ FileSink.forRowFormat(
+ new Path(outputPath + "/result.txt"),
+ new SimpleStringEncoder())
+ .build())
.setParallelism(1);
sEnv.execute();
diff --git a/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java b/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java
index 27d90973a8d66..896d115b4f3bb 100644
--- a/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java
+++ b/flink-end-to-end-tests/flink-confluent-schema-registry/src/main/java/org/apache/flink/schema/registry/test/TestAvroConsumerConfluent.java
@@ -21,7 +21,6 @@
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.serialization.SimpleStringSchema;
import org.apache.flink.api.connector.sink2.Sink;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.connector.kafka.sink.KafkaRecordSerializationSchema;
import org.apache.flink.connector.kafka.sink.KafkaSink;
import org.apache.flink.connector.kafka.source.KafkaSource;
@@ -32,6 +31,7 @@
import org.apache.flink.streaming.api.datastream.DataStreamSource;
import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.ParameterTool;
import example.avro.User;
import org.apache.avro.specific.SpecificRecordBase;
diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml b/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml
deleted file mode 100644
index 298665a1b6892..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-allround-test/pom.xml
+++ /dev/null
@@ -1,70 +0,0 @@
-
-
-
-
- 4.0.0
-
-
- org.apache.flink
- flink-end-to-end-tests
- 2.0-SNAPSHOT
-
-
- flink-dataset-allround-test
- Flink : E2E Tests : Dataset allround
- jar
-
-
-
- org.apache.flink
- flink-java
- ${project.version}
- provided
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-shade-plugin
-
-
- DataSetAllroundTestProgram
- package
-
- shade
-
-
- DataSetAllroundTestProgram
-
-
- org.apache.flink.batch.tests.DataSetAllroundTestProgram
-
-
-
-
-
-
-
-
-
-
diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java b/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java
deleted file mode 100644
index 565fa8238f919..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/DataSetAllroundTestProgram.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.batch.tests;
-
-import org.apache.flink.api.common.functions.CoGroupFunction;
-import org.apache.flink.api.common.functions.FlatMapFunction;
-import org.apache.flink.api.common.functions.GroupReduceFunction;
-import org.apache.flink.api.common.operators.Order;
-import org.apache.flink.api.common.operators.base.JoinOperatorBase;
-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.api.java.functions.KeySelector;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.core.fs.FileSystem;
-
-/**
- * Program to test a large chunk of DataSet API operators and primitives:
- *
- *
- *
Map, FlatMap, Filter
- *
GroupReduce, Reduce
- *
Join
- *
CoGroup
- *
BulkIteration
- *
Different key definitions (position, name, KeySelector)
- *
- *
- *
Program parameters:
- *
- *
- *
loadFactor (int): controls generated data volume. Does not affect result.
- *
outputPath (String): path to write the result
- *
infinite (Boolean): if set to true one of the sources will be infinite. The job will never
- * end. (default: false(
- *
- */
-public class DataSetAllroundTestProgram {
-
- @SuppressWarnings("Convert2Lambda")
- public static void main(String[] args) throws Exception {
-
- // get parameters
- ParameterTool params = ParameterTool.fromArgs(args);
- int loadFactor = Integer.parseInt(params.getRequired("loadFactor"));
- String outputPath = params.getRequired("outputPath");
- boolean infinite = params.getBoolean("infinite", false);
-
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- int numKeys = loadFactor * 128 * 1024;
- DataSet> x1Keys;
- DataSet> x2Keys =
- env.createInput(Generator.generate(numKeys * 32, 2)).setParallelism(4);
- DataSet> x8Keys =
- env.createInput(Generator.generate(numKeys, 8)).setParallelism(4);
-
- if (infinite) {
- x1Keys = env.createInput(Generator.generateInfinitely(numKeys)).setParallelism(4);
- } else {
- x1Keys = env.createInput(Generator.generate(numKeys, 1)).setParallelism(4);
- }
-
- DataSet> joined =
- x2Keys
- // shift keys (check for correct handling of key positions)
- .map(x -> Tuple4.of("0-0", 0L, 1, x.f0))
- .returns(Types.TUPLE(Types.STRING, Types.LONG, Types.INT, Types.STRING))
- // join datasets on non-unique fields (m-n join)
- // Result: (key, 1) 16 * #keys records, all keys are preserved
- .join(x8Keys)
- .where(3)
- .equalTo(0)
- .with((l, r) -> Tuple2.of(l.f3, 1))
- .returns(Types.TUPLE(Types.STRING, Types.INT))
- // key definition with key selector function
- .groupBy(
- new KeySelector, String>() {
- @Override
- public String getKey(Tuple2 value) {
- return value.f0;
- }
- })
- // reduce
- // Result: (key, cnt), #keys records with unique keys, cnt = 16
- .reduce((value1, value2) -> Tuple2.of(value1.f0, value1.f1 + value2.f1));
-
- // co-group two datasets on their primary keys.
- // we filter both inputs such that only 6.25% of the keys overlap.
- // result: (key, cnt), #keys records with unique keys, cnt = (6.25%: 2, 93.75%: 1)
- DataSet> coGrouped =
- x1Keys.filter(x -> x.f1 > 59)
- .coGroup(x1Keys.filter(x -> x.f1 < 68))
- .where("f0")
- .equalTo("f0")
- .with(
- (CoGroupFunction<
- Tuple2,
- Tuple2,
- Tuple2>)
- (l, r, out) -> {
- int cnt = 0;
- String key = "";
- for (Tuple2 t : l) {
- cnt++;
- key = t.f0;
- }
- for (Tuple2 t : r) {
- cnt++;
- key = t.f0;
- }
- out.collect(Tuple2.of(key, cnt));
- })
- .returns(Types.TUPLE(Types.STRING, Types.INT));
-
- // join datasets on keys (1-1 join) and replicate by 16 (previously computed count)
- // result: (key, cnt), 16 * #keys records, all keys preserved, cnt = (6.25%: 2, 93.75%: 1)
- DataSet> joined2 =
- joined.join(coGrouped, JoinOperatorBase.JoinHint.REPARTITION_SORT_MERGE)
- .where(0)
- .equalTo("f0")
- .flatMap(
- (FlatMapFunction<
- Tuple2<
- Tuple2,
- Tuple2>,
- Tuple2>)
- (p, out) -> {
- for (int i = 0; i < p.f0.f1; i++) {
- out.collect(Tuple2.of(p.f0.f0, p.f1.f1));
- }
- })
- .returns(Types.TUPLE(Types.STRING, Types.INT));
-
- // iteration. double the count field until all counts are at 32 or more
- // result: (key, cnt), 16 * #keys records, all keys preserved, cnt = (6.25%: 64, 93.75%: 32)
- IterativeDataSet> initial = joined2.iterate(16);
- DataSet> iteration =
- initial.map(x -> Tuple2.of(x.f0, x.f1 * 2))
- .returns(Types.TUPLE(Types.STRING, Types.INT));
- DataSet termination =
- iteration
- // stop iteration if all values are larger/equal 32
- .flatMap(
- (FlatMapFunction, Boolean>)
- (x, out) -> {
- if (x.f1 < 32) {
- out.collect(false);
- }
- })
- .returns(Types.BOOLEAN);
- DataSet> result =
- initial.closeWith(iteration, termination)
- // group on the count field and count records
- // result: two records: (32, cnt1) and (64, cnt2) where cnt1 = x * 15/16,
- // cnt2 = x * 1/16
- .groupBy(1)
- .reduceGroup(
- (GroupReduceFunction<
- Tuple2, Tuple2>)
- (g, out) -> {
- int key = 0;
- int cnt = 0;
- for (Tuple2 r : g) {
- key = r.f1;
- cnt++;
- }
- out.collect(Tuple2.of(key, cnt));
- })
- .returns(Types.TUPLE(Types.INT, Types.INT))
- // normalize result by load factor
- // result: two records: (32: 15360) and (64, 1024). (x = 16384)
- .map(x -> Tuple2.of(x.f0, x.f1 / (loadFactor * 128)))
- .returns(Types.TUPLE(Types.INT, Types.INT));
-
- // sort and emit result
- result.sortPartition(0, Order.ASCENDING)
- .setParallelism(1)
- .writeAsText(outputPath, FileSystem.WriteMode.OVERWRITE)
- .setParallelism(1);
-
- env.execute();
- }
-}
diff --git a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/Generator.java b/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/Generator.java
deleted file mode 100644
index bdb2ecb963d53..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-allround-test/src/main/java/org/apache/flink/batch/tests/Generator.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.batch.tests;
-
-import org.apache.flink.api.common.io.DefaultInputSplitAssigner;
-import org.apache.flink.api.common.io.InputFormat;
-import org.apache.flink.api.common.io.statistics.BaseStatistics;
-import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.core.io.GenericInputSplit;
-import org.apache.flink.core.io.InputSplitAssigner;
-import org.apache.flink.util.Preconditions;
-
-import java.io.IOException;
-
-/**
- * InputFormat that generates a deterministic DataSet of Tuple2(String, Integer)
- *
- *
- *
String: key, can be repeated.
- *
Integer: uniformly distributed int between 0 and 127
- *
- */
-public class Generator implements InputFormat, GenericInputSplit> {
-
- // total number of records
- private final long numRecords;
- // total number of keys
- private final long numKeys;
-
- // records emitted per partition
- private long recordsPerPartition;
- // number of keys per partition
- private long keysPerPartition;
-
- // number of currently emitted records
- private long recordCnt;
-
- // id of current partition
- private int partitionId;
-
- private final boolean infinite;
-
- public static Generator generate(long numKeys, int recordsPerKey) {
- return new Generator(numKeys, recordsPerKey, false);
- }
-
- public static Generator generateInfinitely(long numKeys) {
- return new Generator(numKeys, 0, true);
- }
-
- private Generator(long numKeys, int recordsPerKey, boolean infinite) {
- this.numKeys = numKeys;
- if (infinite) {
- this.numRecords = Long.MAX_VALUE;
- } else {
- this.numRecords = numKeys * recordsPerKey;
- }
- this.infinite = infinite;
- }
-
- @Override
- public void configure(Configuration parameters) {}
-
- @Override
- public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
- return null;
- }
-
- @Override
- public GenericInputSplit[] createInputSplits(int minNumSplits) {
-
- GenericInputSplit[] splits = new GenericInputSplit[minNumSplits];
- for (int i = 0; i < minNumSplits; i++) {
- splits[i] = new GenericInputSplit(i, minNumSplits);
- }
- return splits;
- }
-
- @Override
- public InputSplitAssigner getInputSplitAssigner(GenericInputSplit[] inputSplits) {
- return new DefaultInputSplitAssigner(inputSplits);
- }
-
- @Override
- public void open(GenericInputSplit split) throws IOException {
- this.partitionId = split.getSplitNumber();
- // total number of partitions
- int numPartitions = split.getTotalNumberOfSplits();
-
- // ensure even distribution of records and keys
- Preconditions.checkArgument(
- numRecords % numPartitions == 0,
- "Records cannot be evenly distributed among partitions");
- Preconditions.checkArgument(
- numKeys % numPartitions == 0, "Keys cannot be evenly distributed among partitions");
-
- this.recordsPerPartition = numRecords / numPartitions;
- this.keysPerPartition = numKeys / numPartitions;
-
- this.recordCnt = 0;
- }
-
- @Override
- public boolean reachedEnd() {
- return !infinite && this.recordCnt >= this.recordsPerPartition;
- }
-
- @Override
- public Tuple2 nextRecord(Tuple2 reuse) throws IOException {
-
- // build key from partition id and count per partition
- String key =
- String.format("%d-%d", this.partitionId, this.recordCnt % this.keysPerPartition);
-
- // 128 values to filter on
- int filterVal = (int) this.recordCnt % 128;
-
- this.recordCnt++;
-
- reuse.f0 = key;
- reuse.f1 = filterVal;
- return reuse;
- }
-
- @Override
- public void close() {}
-}
diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml
deleted file mode 100644
index 2160f12fee596..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/pom.xml
+++ /dev/null
@@ -1,88 +0,0 @@
-
-
-
-
- 4.0.0
-
-
- org.apache.flink
- flink-end-to-end-tests
- 2.0-SNAPSHOT
-
-
- flink-dataset-fine-grained-recovery-test
- Flink : E2E Tests : Dataset Fine-grained recovery
- jar
-
-
-
- org.apache.flink
- flink-java
- ${project.version}
- provided
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-surefire-plugin
-
-
- dataset-fine-grained-recovery
- test
-
- test
-
-
-
- **/*Test.*
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-shade-plugin
-
-
- DataSetFineGrainedRecoveryTestProgram
- package
-
- shade
-
-
- DataSetFineGrainedRecoveryTestProgram
-
-
- org.apache.flink.batch.tests.DataSetFineGrainedRecoveryTestProgram
-
-
-
-
-
-
-
-
-
-
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
deleted file mode 100644
index b14e1467e8afb..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/BlockingIncrementingMapFunction.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.flink.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 java.nio.file.Paths;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A map function that increments values by one.
- *
- *
Processing of elements is held until a latch file is created.
- */
-public class BlockingIncrementingMapFunction extends RichMapFunction {
-
- private final String latchFilePath;
-
- private transient FileBasedOneShotLatch latch;
-
- public BlockingIncrementingMapFunction(final String latchFilePath) {
- this.latchFilePath = checkNotNull(latchFilePath);
- }
-
- @Override
- public void open(final OpenContext openContext) {
- latch = new FileBasedOneShotLatch(Paths.get(latchFilePath));
- }
-
- @Override
- public void close() throws Exception {
- latch.close();
- }
-
- @Override
- public Long map(final Long value) throws InterruptedException {
- latch.await();
- return value + 1;
- }
-}
diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/DataSetFineGrainedRecoveryTestProgram.java b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/DataSetFineGrainedRecoveryTestProgram.java
deleted file mode 100644
index 28578021a3f4e..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/DataSetFineGrainedRecoveryTestProgram.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.flink.batch.tests;
-
-import org.apache.flink.api.common.ExecutionMode;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.core.fs.FileSystem;
-
-/** Program to test fine grained recovery. */
-public class DataSetFineGrainedRecoveryTestProgram {
-
- public static void main(String[] args) throws Exception {
- final ParameterTool params = ParameterTool.fromArgs(args);
- final String latchFilePath = params.getRequired("latchFilePath");
- final String outputPath = params.getRequired("outputPath");
-
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().setExecutionMode(ExecutionMode.BATCH_FORCED);
- env.setParallelism(4);
-
- env.generateSequence(0, 1000)
- .map(new BlockingIncrementingMapFunction(latchFilePath))
- .writeAsText(outputPath, FileSystem.WriteMode.OVERWRITE)
- .setParallelism(1);
-
- env.execute();
- }
-}
diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/util/FileBasedOneShotLatch.java b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/util/FileBasedOneShotLatch.java
deleted file mode 100644
index c41b9479d846e..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/main/java/org/apache/flink/batch/tests/util/FileBasedOneShotLatch.java
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.flink.batch.tests.util;
-
-import com.sun.nio.file.SensitivityWatchEventModifier;
-
-import javax.annotation.concurrent.NotThreadSafe;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.StandardWatchEventKinds;
-import java.nio.file.WatchEvent;
-import java.nio.file.WatchKey;
-import java.nio.file.WatchService;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * A synchronization aid that allows a single thread to wait on the creation of a specified file.
- */
-@NotThreadSafe
-public class FileBasedOneShotLatch implements Closeable {
-
- private final Path latchFile;
-
- private final WatchService watchService;
-
- private boolean released;
-
- public FileBasedOneShotLatch(final Path latchFile) {
- this.latchFile = checkNotNull(latchFile);
-
- final Path parentDir = checkNotNull(latchFile.getParent(), "latchFile must have a parent");
- this.watchService = initWatchService(parentDir);
- }
-
- private static WatchService initWatchService(final Path parentDir) {
- final WatchService watchService = createWatchService(parentDir);
- watchForLatchFile(watchService, parentDir);
- return watchService;
- }
-
- private static WatchService createWatchService(final Path parentDir) {
- try {
- return parentDir.getFileSystem().newWatchService();
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- private static void watchForLatchFile(final WatchService watchService, final Path parentDir) {
- try {
- parentDir.register(
- watchService,
- new WatchEvent.Kind[] {StandardWatchEventKinds.ENTRY_CREATE},
- SensitivityWatchEventModifier.HIGH);
- } catch (IOException e) {
- throw new RuntimeException(e);
- }
- }
-
- /**
- * Waits until the latch file is created.
- *
- *
When this method returns, subsequent invocations will not block even after the latch file
- * is deleted. Note that this method may not return if the latch file is deleted before this
- * method returns.
- *
- * @throws InterruptedException if interrupted while waiting
- */
- public void await() throws InterruptedException {
- if (isReleasedOrReleasable()) {
- return;
- }
-
- awaitLatchFile(watchService);
- }
-
- private void awaitLatchFile(final WatchService watchService) throws InterruptedException {
- while (true) {
- WatchKey watchKey = watchService.take();
- if (isReleasedOrReleasable()) {
- break;
- }
- watchKey.reset();
- }
- }
-
- private boolean isReleasedOrReleasable() {
- if (released) {
- return true;
- }
-
- if (Files.exists(latchFile)) {
- releaseLatch();
- return true;
- }
-
- return false;
- }
-
- private void releaseLatch() {
- released = true;
- }
-
- @Override
- public void close() throws IOException {
- watchService.close();
- }
-}
diff --git a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/test/java/org/apache/flink/batch/tests/util/FileBasedOneShotLatchTest.java b/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/test/java/org/apache/flink/batch/tests/util/FileBasedOneShotLatchTest.java
deleted file mode 100644
index f0d79526c97d1..0000000000000
--- a/flink-end-to-end-tests/flink-dataset-fine-grained-recovery-test/src/test/java/org/apache/flink/batch/tests/util/FileBasedOneShotLatchTest.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.flink.batch.tests.util;
-
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static org.junit.Assert.assertTrue;
-
-/** Tests for {@link FileBasedOneShotLatch}. */
-public class FileBasedOneShotLatchTest {
-
- @Rule public final TemporaryFolder temporaryFolder = new TemporaryFolder();
-
- private FileBasedOneShotLatch latch;
-
- private File latchFile;
-
- @Before
- public void setUp() {
- latchFile = new File(temporaryFolder.getRoot(), "latchFile");
- latch = new FileBasedOneShotLatch(latchFile.toPath());
- }
-
- @Test
- public void awaitReturnsWhenFileIsCreated() throws Exception {
- final AtomicBoolean awaitCompleted = new AtomicBoolean();
- final Thread thread =
- new Thread(
- () -> {
- try {
- latch.await();
- awaitCompleted.set(true);
- } catch (InterruptedException e) {
- Thread.currentThread().interrupt();
- }
- });
- thread.start();
-
- latchFile.createNewFile();
- thread.join();
-
- assertTrue(awaitCompleted.get());
- }
-
- @Test
- public void subsequentAwaitDoesNotBlock() throws Exception {
- latchFile.createNewFile();
- latch.await();
- latch.await();
- }
-
- @Test
- public void subsequentAwaitDoesNotBlockEvenIfLatchFileIsDeleted() throws Exception {
- latchFile.createNewFile();
- latch.await();
-
- latchFile.delete();
- latch.await();
- }
-
- @Test
- public void doesNotBlockIfFileExistsPriorToCreatingLatch() throws Exception {
- latchFile.createNewFile();
-
- final FileBasedOneShotLatch latch = new FileBasedOneShotLatch(latchFile.toPath());
- latch.await();
- }
-}
diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java
index 470cdcefc2c4c..1720004be2ee6 100644
--- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java
+++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestJobFactory.java
@@ -28,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.ResultTypeQueryable;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
@@ -51,6 +50,7 @@
import org.apache.flink.streaming.tests.artificialstate.builder.ArtificialListStateBuilder;
import org.apache.flink.streaming.tests.artificialstate.builder.ArtificialStateBuilder;
import org.apache.flink.streaming.tests.artificialstate.builder.ArtificialValueStateBuilder;
+import org.apache.flink.util.ParameterTool;
import java.io.IOException;
import java.time.Duration;
diff --git a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java
index a30d4b3c63257..bd3e991a43087 100644
--- a/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java
+++ b/flink-end-to-end-tests/flink-datastream-allround-test/src/main/java/org/apache/flink/streaming/tests/DataStreamAllroundTestProgram.java
@@ -22,7 +22,6 @@
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.formats.avro.typeutils.AvroSerializer;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -34,6 +33,7 @@
import org.apache.flink.streaming.tests.avro.ComplexPayloadAvro;
import org.apache.flink.streaming.tests.avro.InnerPayLoadAvro;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.ParameterTool;
import java.util.Arrays;
import java.util.Collections;
diff --git a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml
index f08ea0d4eb866..f63af10b9bfff 100644
--- a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml
+++ b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/pom.xml
@@ -40,7 +40,13 @@
${project.version}provided
-
+
+ org.apache.flink
+ flink-connector-files
+ 2.0-SNAPSHOT
+ provided
+
+
diff --git a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/src/main/java/org/apache/flink/streaming/tests/DistributedCacheViaBlobTestProgram.java b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/src/main/java/org/apache/flink/streaming/tests/DistributedCacheViaBlobTestProgram.java
index 204e1382053a9..9e3c9eb9ae509 100644
--- a/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/src/main/java/org/apache/flink/streaming/tests/DistributedCacheViaBlobTestProgram.java
+++ b/flink-end-to-end-tests/flink-distributed-cache-via-blob-test/src/main/java/org/apache/flink/streaming/tests/DistributedCacheViaBlobTestProgram.java
@@ -18,9 +18,10 @@
package org.apache.flink.streaming.tests;
import org.apache.flink.api.common.functions.RichMapFunction;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.ParameterTool;
import java.nio.file.Files;
import java.nio.file.Path;
@@ -65,7 +66,12 @@ public static void main(String[] args) throws Exception {
Files.size(inputFile),
inputDir.toAbsolutePath().toString(),
containedFile.getFileName().toString()))
- .writeAsText(params.getRequired("output"), FileSystem.WriteMode.OVERWRITE);
+ .sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(
+ params.getRequired("output")),
+ new SimpleStringEncoder())
+ .build());
env.execute("Distributed Cache Via Blob Test Program");
}
diff --git a/flink-end-to-end-tests/flink-failure-enricher-test/pom.xml b/flink-end-to-end-tests/flink-failure-enricher-test/pom.xml
index 028ae2b30a4e4..b4a08ecae4bf2 100644
--- a/flink-end-to-end-tests/flink-failure-enricher-test/pom.xml
+++ b/flink-end-to-end-tests/flink-failure-enricher-test/pom.xml
@@ -54,7 +54,13 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
${project.version}provided
-
+
+ org.apache.flink
+ flink-connector-files
+ ${project.version}
+ provided
+
+
diff --git a/flink-end-to-end-tests/flink-failure-enricher-test/src/main/java/org/apache/flink/streaming/tests/FailureEnricherTestProgram.java b/flink-end-to-end-tests/flink-failure-enricher-test/src/main/java/org/apache/flink/streaming/tests/FailureEnricherTestProgram.java
index 5e63f9b5fd73d..659752da9042e 100644
--- a/flink-end-to-end-tests/flink-failure-enricher-test/src/main/java/org/apache/flink/streaming/tests/FailureEnricherTestProgram.java
+++ b/flink-end-to-end-tests/flink-failure-enricher-test/src/main/java/org/apache/flink/streaming/tests/FailureEnricherTestProgram.java
@@ -18,9 +18,10 @@
package org.apache.flink.streaming.tests;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.core.fs.FileSystem;
+import org.apache.flink.api.common.serialization.SimpleStringEncoder;
+import org.apache.flink.connector.file.sink.FileSink;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.util.ParameterTool;
/**
* End-to-end test program for verifying that the {@link
@@ -43,7 +44,12 @@ public static void main(String[] args) throws Exception {
value -> {
throw new RuntimeException("Expect exception");
})
- .writeAsText(params.getRequired("output"), FileSystem.WriteMode.OVERWRITE);
+ .sinkTo(
+ FileSink.forRowFormat(
+ new org.apache.flink.core.fs.Path(
+ params.getRequired("output")),
+ new SimpleStringEncoder())
+ .build());
env.execute("Failure Enricher Test");
}
diff --git a/flink-end-to-end-tests/flink-file-sink-test/src/main/java/org/apache/flink/connector/file/sink/FileSinkProgram.java b/flink-end-to-end-tests/flink-file-sink-test/src/main/java/org/apache/flink/connector/file/sink/FileSinkProgram.java
index d57cd9e8323d0..ef92622bde608 100644
--- a/flink-end-to-end-tests/flink-file-sink-test/src/main/java/org/apache/flink/connector/file/sink/FileSinkProgram.java
+++ b/flink-end-to-end-tests/flink-file-sink-test/src/main/java/org/apache/flink/connector/file/sink/FileSinkProgram.java
@@ -23,7 +23,6 @@
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.api.java.tuple.Tuple2;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.RestartStrategyOptions;
import org.apache.flink.core.fs.Path;
@@ -38,6 +37,7 @@
import org.apache.flink.streaming.api.functions.sink.filesystem.legacy.StreamingFileSink;
import org.apache.flink.streaming.api.functions.sink.filesystem.rollingpolicies.OnCheckpointRollingPolicy;
import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction;
+import org.apache.flink.util.ParameterTool;
import java.io.PrintStream;
import java.time.Duration;
diff --git a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java
index d8631f29f7408..da25c271daf84 100644
--- a/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java
+++ b/flink-end-to-end-tests/flink-heavy-deployment-stress-test/src/main/java/org/apache/flink/deployment/HeavyDeploymentStressTestProgram.java
@@ -23,7 +23,6 @@
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.state.OperatorStateStore;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.runtime.state.FunctionInitializationContext;
@@ -32,6 +31,7 @@
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.legacy.RichParallelSourceFunction;
+import org.apache.flink.util.ParameterTool;
import org.apache.flink.util.Preconditions;
import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.setupEnvironment;
diff --git a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java
index 2accdd879b950..15bb8c2e7f62c 100644
--- a/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java
+++ b/flink-end-to-end-tests/flink-local-recovery-and-allocation-test/src/main/java/org/apache/flink/streaming/tests/StickyAllocationAndLocalRecoveryTestJob.java
@@ -26,7 +26,6 @@
import org.apache.flink.api.common.state.ValueState;
import org.apache.flink.api.common.state.ValueStateDescriptor;
import org.apache.flink.api.java.functions.KeySelector;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.ExternalizedCheckpointRetention;
@@ -40,6 +39,7 @@
import org.apache.flink.streaming.api.functions.source.legacy.RichParallelSourceFunction;
import org.apache.flink.streaming.api.operators.StreamingRuntimeContext;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.ParameterTool;
import org.apache.flink.util.Preconditions;
import org.apache.commons.lang3.RandomStringUtils;
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 8231c9ecc1acb..27799005cd3a0 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
@@ -19,12 +19,12 @@
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.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.legacy.RichSinkFunction;
import org.apache.flink.streaming.api.functions.source.legacy.RichParallelSourceFunction;
+import org.apache.flink.util.ParameterTool;
import org.apache.flink.shaded.netty4.io.netty.util.internal.OutOfDirectMemoryError;
diff --git a/flink-end-to-end-tests/flink-parent-child-classloading-test-program/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java b/flink-end-to-end-tests/flink-parent-child-classloading-test-program/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java
index ac36e84056578..ed62ccc1cc3a1 100644
--- a/flink-end-to-end-tests/flink-parent-child-classloading-test-program/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java
+++ b/flink-end-to-end-tests/flink-parent-child-classloading-test-program/src/main/java/org/apache/flink/streaming/tests/ClassLoaderTestProgram.java
@@ -18,9 +18,9 @@
package org.apache.flink.streaming.tests;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
+import org.apache.flink.util.ParameterTool;
import java.io.InputStream;
import java.net.URL;
@@ -88,7 +88,7 @@ public static void main(String[] args) throws Exception {
+ ":"
+ orderedProperties;
})
- .writeAsText(params.getRequired("output"), FileSystem.WriteMode.OVERWRITE);
+ .sinkTo(new DiscardingSink<>());
env.execute("ClassLoader Test Program");
}
diff --git a/flink-end-to-end-tests/flink-python-test/pom.xml b/flink-end-to-end-tests/flink-python-test/pom.xml
index fb4b1d2fd562f..54485a0c1ee03 100644
--- a/flink-end-to-end-tests/flink-python-test/pom.xml
+++ b/flink-end-to-end-tests/flink-python-test/pom.xml
@@ -37,12 +37,6 @@
${project.version}provided
-
- org.apache.flink
- flink-java
- ${project.version}
- provided
- org.apache.flinkflink-streaming-java
diff --git a/flink-end-to-end-tests/flink-queryable-state-test/pom.xml b/flink-end-to-end-tests/flink-queryable-state-test/pom.xml
index 30e741c52bcaf..8615fca1383ef 100644
--- a/flink-end-to-end-tests/flink-queryable-state-test/pom.xml
+++ b/flink-end-to-end-tests/flink-queryable-state-test/pom.xml
@@ -39,12 +39,6 @@
${project.version}
-
- org.apache.flink
- flink-java
- ${project.version}
-
- org.apache.flinkflink-queryable-state-client-java
diff --git a/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateClient.java b/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateClient.java
index 10eaa0dfd039b..1ef7a60e9fd8f 100644
--- a/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateClient.java
+++ b/flink-end-to-end-tests/flink-queryable-state-test/src/main/java/org/apache/flink/streaming/tests/queryablestate/QsStateClient.java
@@ -24,9 +24,9 @@
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.common.typeinfo.TypeHint;
import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.queryablestate.client.QueryableStateClient;
import org.apache.flink.queryablestate.exceptions.UnknownKeyOrNamespaceException;
+import org.apache.flink.util.ParameterTool;
import java.util.Map;
import java.util.concurrent.CompletableFuture;
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 1d9387a81c177..e80fcb6183556 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
@@ -25,7 +25,6 @@
import org.apache.flink.api.common.typeinfo.TypeHint;
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.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.StateBackendOptions;
@@ -36,6 +35,7 @@
import org.apache.flink.streaming.api.functions.source.legacy.RichSourceFunction;
import org.apache.flink.streaming.api.functions.source.legacy.SourceFunction;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.ParameterTool;
import org.apache.flink.shaded.guava32.com.google.common.collect.Iterables;
diff --git a/flink-end-to-end-tests/flink-quickstart-test/src/main/java/org/apache/flink/quickstarts/test/QuickstartExample.java b/flink-end-to-end-tests/flink-quickstart-test/src/main/java/org/apache/flink/quickstarts/test/QuickstartExample.java
index 1ca4b7c747251..de197b244dcd9 100644
--- a/flink-end-to-end-tests/flink-quickstart-test/src/main/java/org/apache/flink/quickstarts/test/QuickstartExample.java
+++ b/flink-end-to-end-tests/flink-quickstart-test/src/main/java/org/apache/flink/quickstarts/test/QuickstartExample.java
@@ -19,11 +19,11 @@
package org.apache.flink.quickstarts.test;
import org.apache.flink.api.common.functions.MapFunction;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.quickstarts.test.utils.Utils;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.ParameterTool;
/** End to end test for quickstarts. */
public class QuickstartExample {
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 b6918c656f162..f984000d4f653 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
@@ -28,12 +28,12 @@
import org.apache.flink.api.common.state.ValueStateDescriptor;
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.streaming.api.datastream.KeyedStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.windowing.WindowFunction;
import org.apache.flink.streaming.api.windowing.windows.TimeWindow;
import org.apache.flink.util.Collector;
+import org.apache.flink.util.ParameterTool;
import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.applyTumblingWindows;
import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.createTimestampExtractor;
diff --git a/flink-end-to-end-tests/flink-state-evolution-test/pom.xml b/flink-end-to-end-tests/flink-state-evolution-test/pom.xml
index faf66d5320640..ec330fb42c92d 100644
--- a/flink-end-to-end-tests/flink-state-evolution-test/pom.xml
+++ b/flink-end-to-end-tests/flink-state-evolution-test/pom.xml
@@ -31,12 +31,6 @@ under the License.
jar
-
- org.apache.flink
- flink-java
- ${project.version}
- provided
- org.apache.flinkflink-streaming-java
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 15faf168fbbc9..49be167927584 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
@@ -25,7 +25,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.api.java.utils.ParameterTool;
import org.apache.flink.avro.generated.Address;
import org.apache.flink.configuration.CheckpointingOptions;
import org.apache.flink.configuration.Configuration;
@@ -36,6 +35,7 @@
import org.apache.flink.streaming.api.functions.sink.v2.DiscardingSink;
import org.apache.flink.streaming.api.functions.source.legacy.RichParallelSourceFunction;
import org.apache.flink.types.Either;
+import org.apache.flink.util.ParameterTool;
import java.io.IOException;
import java.util.Objects;
diff --git a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java
index 782a4c12a6cb7..4d2ab63de47fd 100644
--- a/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java
+++ b/flink-end-to-end-tests/flink-stream-sql-test/src/main/java/org/apache/flink/sql/tests/StreamSQLTestProgram.java
@@ -27,7 +27,6 @@
import org.apache.flink.api.common.typeutils.base.IntSerializer;
import org.apache.flink.api.common.typeutils.base.LongSerializer;
import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.configuration.RestartStrategyOptions;
import org.apache.flink.core.fs.Path;
@@ -54,6 +53,7 @@
import org.apache.flink.table.legacy.sources.tsextractors.ExistingField;
import org.apache.flink.table.sources.wmstrategies.BoundedOutOfOrderTimestamps;
import org.apache.flink.types.Row;
+import org.apache.flink.util.ParameterTool;
import java.io.PrintStream;
import java.time.Duration;
diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
index e337a0a276abf..d122e8a22a526 100644
--- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
+++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/DataStreamStateTTLTestProgram.java
@@ -19,13 +19,13 @@
package org.apache.flink.streaming.tests;
import org.apache.flink.api.common.state.StateTtlConfig;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.contrib.streaming.state.EmbeddedRocksDBStateBackend;
import org.apache.flink.runtime.state.StateBackend;
import org.apache.flink.runtime.state.hashmap.HashMapStateBackend;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.legacy.PrintSinkFunction;
import org.apache.flink.streaming.api.graph.StreamGraph;
+import org.apache.flink.util.ParameterTool;
import static org.apache.flink.configuration.StateBackendOptions.STATE_BACKEND;
import static org.apache.flink.streaming.tests.DataStreamAllroundTestJobFactory.STATE_BACKEND_ROCKS_INCREMENTAL;
diff --git a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlTestConfig.java b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlTestConfig.java
index 8ebda54add349..aad2fbf370b32 100644
--- a/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlTestConfig.java
+++ b/flink-end-to-end-tests/flink-stream-state-ttl-test/src/main/java/org/apache/flink/streaming/tests/TtlTestConfig.java
@@ -18,9 +18,9 @@
package org.apache.flink.streaming.tests;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.util.ParameterTool;
import java.time.Duration;
diff --git a/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/src/main/java/org/apache/flink/streaming/tests/StatefulStreamJobUpgradeTestProgram.java b/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/src/main/java/org/apache/flink/streaming/tests/StatefulStreamJobUpgradeTestProgram.java
index 714c4b5687d41..349ba6efdc418 100644
--- a/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/src/main/java/org/apache/flink/streaming/tests/StatefulStreamJobUpgradeTestProgram.java
+++ b/flink-end-to-end-tests/flink-stream-stateful-job-upgrade-test/src/main/java/org/apache/flink/streaming/tests/StatefulStreamJobUpgradeTestProgram.java
@@ -22,13 +22,13 @@
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.configuration.ConfigOption;
import org.apache.flink.configuration.ConfigOptions;
import org.apache.flink.streaming.api.datastream.KeyedStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.legacy.PrintSinkFunction;
import org.apache.flink.streaming.tests.artificialstate.ComplexPayload;
+import org.apache.flink.util.ParameterTool;
import java.util.Collections;
import java.util.List;
diff --git a/flink-end-to-end-tests/flink-tpcds-test/pom.xml b/flink-end-to-end-tests/flink-tpcds-test/pom.xml
index 1a231b9e83966..acde5faa3a033 100644
--- a/flink-end-to-end-tests/flink-tpcds-test/pom.xml
+++ b/flink-end-to-end-tests/flink-tpcds-test/pom.xml
@@ -35,12 +35,6 @@ under the License.
${project.version}provided
-
- org.apache.flink
- flink-java
- ${project.version}
- provided
- org.apache.flinkflink-streaming-java
diff --git a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
index 8ebf202afcb85..9ad901de8fdae 100644
--- a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
+++ b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/TpcdsTestProgram.java
@@ -18,7 +18,6 @@
package org.apache.flink.table.tpcds;
-import org.apache.flink.api.java.utils.ParameterTool;
import org.apache.flink.core.fs.FileSystem;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.Table;
@@ -34,6 +33,7 @@
import org.apache.flink.table.tpcds.schema.TpcdsSchemaProvider;
import org.apache.flink.table.tpcds.stats.TpcdsStatsProvider;
import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.util.ParameterTool;
import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
diff --git a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/AnswerFormatter.java b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/AnswerFormatter.java
index aa1b962caa301..a97b8d621395c 100644
--- a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/AnswerFormatter.java
+++ b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/AnswerFormatter.java
@@ -18,7 +18,7 @@
package org.apache.flink.table.tpcds.utils;
-import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.util.ParameterTool;
import java.io.BufferedReader;
import java.io.BufferedWriter;
diff --git a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/TpcdsResultComparator.java b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/TpcdsResultComparator.java
index bd78c52b28a52..2d872c50e787a 100644
--- a/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/TpcdsResultComparator.java
+++ b/flink-end-to-end-tests/flink-tpcds-test/src/main/java/org/apache/flink/table/tpcds/utils/TpcdsResultComparator.java
@@ -18,7 +18,7 @@
package org.apache.flink.table.tpcds.utils;
-import org.apache.flink.api.java.utils.ParameterTool;
+import org.apache.flink.util.ParameterTool;
import java.io.File;
import java.nio.charset.StandardCharsets;
diff --git a/flink-end-to-end-tests/pom.xml b/flink-end-to-end-tests/pom.xml
index 1738d60f724e5..4e3ab80a9cf0e 100644
--- a/flink-end-to-end-tests/pom.xml
+++ b/flink-end-to-end-tests/pom.xml
@@ -42,8 +42,6 @@ under the License.
flink-cli-testflink-parent-child-classloading-test-programflink-parent-child-classloading-test-lib-package
- flink-dataset-allround-test
- flink-dataset-fine-grained-recovery-testflink-datastream-allround-testflink-batch-sql-testflink-stream-sql-test
diff --git a/flink-end-to-end-tests/run-nightly-tests.sh b/flink-end-to-end-tests/run-nightly-tests.sh
index 286f293d7a427..97c9b6b8deb39 100755
--- a/flink-end-to-end-tests/run-nightly-tests.sh
+++ b/flink-end-to-end-tests/run-nightly-tests.sh
@@ -153,8 +153,6 @@ function run_group_1 {
# High Availability
################################################################################
- run_test "Running HA dataset end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_dataset.sh" "skip_check_exceptions"
-
run_test "Running HA (hashmap, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh hashmap true false" "skip_check_exceptions"
run_test "Running HA (hashmap, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh hashmap false false" "skip_check_exceptions"
run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha_datastream.sh rocks true false" "skip_check_exceptions"
@@ -178,7 +176,6 @@ function run_group_2 {
run_test "Queryable state (rocksdb) end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state.sh rocksdb"
run_test "Queryable state (rocksdb) with TM restart end-to-end test" "$END_TO_END_DIR/test-scripts/test_queryable_state_restart_tm.sh" "skip_check_exceptions"
- run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
run_test "Streaming SQL end-to-end test using planner loader" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh" "skip_check_exceptions"
run_test "Streaming SQL end-to-end test using planner with Scala version" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh scala-planner" "skip_check_exceptions"
run_test "Sql Jdbc Driver end-to-end test" "$END_TO_END_DIR/test-scripts/test_sql_jdbc_driver.sh" "skip_check_exceptions"
diff --git a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
index c12fa071fe42d..d5599d9ee3c04 100755
--- a/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
+++ b/flink-end-to-end-tests/test-scripts/common_kubernetes.sh
@@ -23,7 +23,7 @@ source "$(dirname "$0")"/common_docker.sh
CONTAINER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts
RETRY_COUNT=3
RETRY_BACKOFF_TIME=5
-RESULT_HASH="e682ec6622b5e83f2eb614617d5ab2cf"
+RESULT_HASH="d41d8cd98f00b204e9800998ecf8427e"
MINIKUBE_VERSION="v1.28.0"
CRICTL_VERSION="v1.24.2"
CRI_DOCKERD_VERSION="0.2.3"
diff --git a/flink-end-to-end-tests/test-scripts/test_azure_fs.sh b/flink-end-to-end-tests/test-scripts/test_azure_fs.sh
index 0ae880e92c341..2d240937e5c95 100755
--- a/flink-end-to-end-tests/test-scripts/test_azure_fs.sh
+++ b/flink-end-to-end-tests/test-scripts/test_azure_fs.sh
@@ -70,6 +70,6 @@ azure_setup
echo "Starting Flink cluster.."
start_cluster
-$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/batch/WordCount.jar --input $AZURE_TEST_DATA_WORDS_URI --output $TEST_DATA_DIR/out/wc_out
-
-check_result_hash "WordCountWithAzureFS" $TEST_DATA_DIR/out/wc_out "72a690412be8928ba239c2da967328a5"
+$FLINK_DIR/bin/flink run -p 1 $FLINK_DIR/examples/streaming/WordCount.jar --input $AZURE_TEST_DATA_WORDS_URI --output $TEST_DATA_DIR/out/wc_out --execution-mode BATCH
+OUTPUT_FILES=$(find "$TEST_DATA_DIR/out/wc_out" -type f)
+check_result_hash "WordCountWithAzureFS" "${OUTPUT_FILES}" "5a9945c9ab08890b2a0f6b31a4437d57"
diff --git a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh b/flink-end-to-end-tests/test-scripts/test_batch_allround.sh
deleted file mode 100755
index f2e05abb2f7db..0000000000000
--- a/flink-end-to-end-tests/test-scripts/test_batch_allround.sh
+++ /dev/null
@@ -1,38 +0,0 @@
-#!/usr/bin/env bash
-################################################################################
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-set -Eeuo pipefail
-
-source "$(dirname "$0")"/common.sh
-
-TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-dataset-allround-test/target/DataSetAllroundTestProgram.jar
-
-echo "Run DataSet-Allround-Test Program"
-
-# modify configuration to include spilling to disk
-set_config_key "taskmanager.memory.network.min" "111149056"
-set_config_key "taskmanager.memory.network.max" "111149056"
-
-set_conf_ssl "server"
-start_cluster
-start_taskmanagers 3
-
-$FLINK_DIR/bin/flink run -p 4 $TEST_PROGRAM_JAR --loadFactor 4 --outputPath $TEST_DATA_DIR/out/dataset_allround
-
-check_result_hash "DataSet-Allround-Test" $TEST_DATA_DIR/out/dataset_allround "d3cf2aeaa9320c772304cba42649eb47"
diff --git a/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh
index 59870f7ee676a..d25a1a74c4ef9 100755
--- a/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh
+++ b/flink-end-to-end-tests/test-scripts/test_batch_wordcount.sh
@@ -20,7 +20,7 @@
source "$(dirname "$0")"/common.sh
INPUT_TYPE=${1:-file}
-RESULT_HASH="72a690412be8928ba239c2da967328a5"
+RESULT_HASH="5a9945c9ab08890b2a0f6b31a4437d57"
S3_PREFIX=temp/test_batch_wordcount-$(uuidgen)
OUTPUT_PATH="${TEST_DATA_DIR}/out/wc_out"
@@ -28,12 +28,12 @@ fetch_complete_result=()
case $INPUT_TYPE in
(file)
- ARGS="--input ${TEST_INFRA_DIR}/test-data/words --output ${OUTPUT_PATH}"
+ ARGS="--execution-mode BATCH --input ${TEST_INFRA_DIR}/test-data/words --output ${OUTPUT_PATH}"
;;
(hadoop)
source "$(dirname "$0")"/common_s3.sh
s3_setup hadoop
- ARGS="--input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
+ ARGS="--execution-mode BATCH --input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
OUTPUT_PATH="$TEST_DATA_DIR/$S3_PREFIX"
on_exit "s3_delete_by_full_path_prefix '$S3_PREFIX'"
fetch_complete_result=(s3_get_by_full_path_and_filename_prefix "$OUTPUT_PATH" "${S3_PREFIX}" "" false)
@@ -41,13 +41,13 @@ case $INPUT_TYPE in
(hadoop_minio)
source "$(dirname "$0")"/common_s3_minio.sh
s3_setup hadoop
- ARGS="--input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
+ ARGS="--execution-mode BATCH --input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
OUTPUT_PATH="$TEST_INFRA_DIR/$IT_CASE_S3_BUCKET/$S3_PREFIX"
;;
(hadoop_with_provider)
source "$(dirname "$0")"/common_s3.sh
s3_setup_with_provider hadoop "fs.s3a.aws.credentials.provider"
- ARGS="--input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
+ ARGS="--execution-mode BATCH --input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
OUTPUT_PATH="$TEST_DATA_DIR/$S3_PREFIX"
on_exit "s3_delete_by_full_path_prefix '$S3_PREFIX'"
fetch_complete_result=(s3_get_by_full_path_and_filename_prefix "$OUTPUT_PATH" "${S3_PREFIX}" "" false)
@@ -55,7 +55,7 @@ case $INPUT_TYPE in
(presto)
source "$(dirname "$0")"/common_s3.sh
s3_setup presto
- ARGS="--input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
+ ARGS="--execution-mode BATCH --input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
OUTPUT_PATH="$TEST_DATA_DIR/$S3_PREFIX"
on_exit "s3_delete_by_full_path_prefix '$S3_PREFIX'"
fetch_complete_result=(s3_get_by_full_path_and_filename_prefix "$OUTPUT_PATH" "${S3_PREFIX}" "" false)
@@ -63,14 +63,14 @@ case $INPUT_TYPE in
(presto_minio)
source "$(dirname "$0")"/common_s3_minio.sh
s3_setup presto
- ARGS="--input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
+ ARGS="--execution-mode BATCH --input ${S3_TEST_DATA_WORDS_URI} --output s3://$IT_CASE_S3_BUCKET/$S3_PREFIX"
OUTPUT_PATH="$TEST_INFRA_DIR/$IT_CASE_S3_BUCKET/$S3_PREFIX"
;;
(dummy-fs)
source "$(dirname "$0")"/common_dummy_fs.sh
dummy_fs_setup
- ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words --output ${OUTPUT_PATH}"
- RESULT_HASH="0e5bd0a3dd7d5a7110aa85ff70adb54b"
+ ARGS="--execution-mode BATCH --input dummy://localhost/words --input anotherDummy://localhost/words --output ${OUTPUT_PATH}"
+ RESULT_HASH="41d097718a0b00f67fe13d21048d1757"
;;
(*)
echo "Unknown input type $INPUT_TYPE"
@@ -83,7 +83,7 @@ start_cluster
# The test may run against different source types.
# But the sources should provide the same test data, so the checksum stays the same for all tests.
-${FLINK_DIR}/bin/flink run -p 1 ${FLINK_DIR}/examples/batch/WordCount.jar ${ARGS}
+${FLINK_DIR}/bin/flink run -p 1 ${FLINK_DIR}/examples/streaming/WordCount.jar ${ARGS}
# Fetches result from AWS s3 to the OUTPUT_PATH, no-op for other filesystems and minio-based tests
# it seems we need a function for retry_times
@@ -91,4 +91,6 @@ function fetch_it() {
${fetch_complete_result[@]}
}
retry_times 10 5 fetch_it
-check_result_hash "WordCount (${INPUT_TYPE})" "${OUTPUT_PATH}" "${RESULT_HASH}"
+
+OUTPUT_FILES=$(find "${OUTPUT_PATH}" -type f)
+check_result_hash "WordCount (${INPUT_TYPE})" "${OUTPUT_FILES}" "${RESULT_HASH}"
diff --git a/flink-end-to-end-tests/test-scripts/test_cli.sh b/flink-end-to-end-tests/test-scripts/test_cli.sh
index 0746469624e5d..b8b86626901da 100755
--- a/flink-end-to-end-tests/test-scripts/test_cli.sh
+++ b/flink-end-to-end-tests/test-scripts/test_cli.sh
@@ -29,8 +29,8 @@ $FLINK_DIR/bin/taskmanager.sh start
$FLINK_DIR/bin/taskmanager.sh start
# CLI regular expressions
-JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR="\"pact\": \"(Data Source)\""
-JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR="\"pact\": \"(Data Sink)\""
+JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR="\"pact\" : \"(Data Source)\""
+JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR="\"pact\" : \"(Data Sink)\""
JOB_LIST_REGEX_EXTRACTOR_BY_STATUS="([0-9,a-f]*) :"
EXIT_CODE=0
@@ -72,11 +72,11 @@ function extract_task_manager_slot_request_count() {
printf "\n==============================================================================\n"
printf "Test default job launch with non-detach mode\n"
printf "==============================================================================\n"
-RESULT=`$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar`
+RESULT=`$FLINK_DIR/bin/flink run $FLINK_DIR/examples/streaming/WordCount.jar --output file:///${TEST_DATA_DIR}/trash`
EXIT_CODE=$?
echo "$RESULT"
-if [[ $RESULT != *"(java.util.ArrayList) [170 elements]"* ]];then
+if [[ $RESULT != *"Job with JobID"*"has finished"* ]];then
echo "[FAIL] Invalid accumulator result."
EXIT_CODE=1
fi
@@ -86,16 +86,16 @@ if [ $EXIT_CODE == 0 ]; then
printf "Test job launch with complex parameter set\n"
printf "==============================================================================\n"
eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 \
- -c org.apache.flink.examples.java.wordcount.WordCount \
- $FLINK_DIR/examples/batch/WordCount.jar \
+ -c org.apache.flink.streaming.examples.wordcount.WordCount \
+ $FLINK_DIR/examples/streaming/WordCount.jar \
--input file:///$FLINK_DIR/README.txt \
--output file:///${TEST_DATA_DIR}/result1"
EXIT_CODE=$?
fi
if [ $EXIT_CODE == 0 ]; then
- ROW_COUNT=`cat ${TEST_DATA_DIR}/result1/* | wc -l`
- if [ $((ROW_COUNT)) -ne 111 ]; then
+ ROW_COUNT=`find ${TEST_DATA_DIR}/result1/* -type f | xargs wc -l | grep "total" | awk '{print $1}'`
+ if [ $((ROW_COUNT)) -ne 192 ]; then
echo "[FAIL] Unexpected number of rows in output."
echo "Found: $ROW_COUNT"
EXIT_CODE=1
@@ -116,7 +116,7 @@ printf "\n======================================================================
printf "Test CLI information\n"
printf "==============================================================================\n"
if [ $EXIT_CODE == 0 ]; then
- RETURN=`$FLINK_DIR/bin/flink info $FLINK_DIR/examples/batch/WordCount.jar`
+ RETURN=`$FLINK_DIR/bin/flink info $FLINK_DIR/examples/streaming/WordCount.jar`
echo "$RETURN"
PACT_MATCH=`extract_valid_pact_from_job_info_return "$RETURN"`
if [[ $PACT_MATCH == -1 ]]; then # expect at least a Data Source and a Data Sink pact match
diff --git a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh
index 92050644255bf..f4a3dad92548f 100755
--- a/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh
+++ b/flink-end-to-end-tests/test-scripts/test_docker_embedded_job.sh
@@ -24,7 +24,7 @@ DOCKER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts
DOCKER_IMAGE_BUILD_RETRIES=3
BUILD_BACKOFF_TIME=5
-export FLINK_JOB=org.apache.flink.examples.java.wordcount.WordCount
+export FLINK_JOB=org.apache.flink.streaming.examples.wordcount.WordCount
export FLINK_DOCKER_IMAGE_NAME=test_docker_embedded_job
export INPUT_VOLUME=${END_TO_END_DIR}/test-scripts/test-data
export OUTPUT_VOLUME=${TEST_DATA_DIR}/out
@@ -32,7 +32,7 @@ export INPUT_PATH=/data/test/input
export OUTPUT_PATH=/data/test/output
INPUT_TYPE=${1:-file}
-RESULT_HASH="72a690412be8928ba239c2da967328a5"
+RESULT_HASH="5a9945c9ab08890b2a0f6b31a4437d57"
case $INPUT_TYPE in
(file)
INPUT_ARGS="--input ${INPUT_PATH}/words"
@@ -41,7 +41,7 @@ case $INPUT_TYPE in
source "$(dirname "$0")"/common_dummy_fs.sh
dummy_fs_setup
INPUT_ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words"
- RESULT_HASH="0e5bd0a3dd7d5a7110aa85ff70adb54b"
+ RESULT_HASH="41d097718a0b00f67fe13d21048d1757"
;;
(*)
echo "Unknown input type $INPUT_TYPE"
@@ -49,7 +49,7 @@ case $INPUT_TYPE in
;;
esac
-export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/docker_wc_out"
+export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/docker_wc_out --execution-mode BATCH"
# user inside the container must be able to create files, this is a workaround in-container permissions
mkdir -p $OUTPUT_VOLUME
@@ -60,10 +60,11 @@ if ! retry_times $DOCKER_IMAGE_BUILD_RETRIES ${BUILD_BACKOFF_TIME} "build_image
exit 1
fi
-export USER_LIB=${FLINK_DIR}/examples/batch
+export USER_LIB=${FLINK_DIR}/examples/streaming
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.test.yml up --force-recreate --abort-on-container-exit --exit-code-from job-cluster &> /dev/null
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.test.yml logs job-cluster > $FLINK_LOG_DIR/jobmanager.log
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.test.yml logs taskmanager > $FLINK_LOG_DIR/taskmanager.log
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.test.yml rm -f
-check_result_hash "WordCount" $OUTPUT_VOLUME/docker_wc_out "${RESULT_HASH}"
+OUTPUT_FILES=$(find "$OUTPUT_VOLUME/docker_wc_out" -type f)
+check_result_hash "WordCount" "${OUTPUT_FILES}" "${RESULT_HASH}"
diff --git a/flink-end-to-end-tests/test-scripts/test_ha_dataset.sh b/flink-end-to-end-tests/test-scripts/test_ha_dataset.sh
deleted file mode 100755
index 1214cee6df204..0000000000000
--- a/flink-end-to-end-tests/test-scripts/test_ha_dataset.sh
+++ /dev/null
@@ -1,100 +0,0 @@
-#!/usr/bin/env bash
-################################################################################
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements. See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership. The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License. You may obtain a copy of the License at
-#
-# http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-################################################################################
-
-source "$(dirname "$0")"/common.sh
-source "$(dirname "$0")"/common_ha.sh
-
-TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-dataset-fine-grained-recovery-test/target/DataSetFineGrainedRecoveryTestProgram.jar
-
-function ha_cleanup() {
- # kill the cluster and zookeeper
- stop_watchdogs
-}
-
-on_exit ha_cleanup
-
-function run_ha_test() {
- local PARALLELISM=$1
-
- local JM_KILLS=2
- local TM_KILLS=2
-
- local LATCH_FILE_PATH=$TEST_DATA_DIR/latchFile
-
- CLEARED=0
-
- setup_and_start_cluster ${PARALLELISM}
- echo "Running on HA mode: parallelism=${PARALLELISM}."
-
- # submit a job in detached mode and let it run
- local JOB_ID=$($FLINK_DIR/bin/flink run -d -p ${PARALLELISM} \
- $TEST_PROGRAM_JAR \
- --latchFilePath $LATCH_FILE_PATH \
- --outputPath $TEST_DATA_DIR/out/dataset_allround \
- | grep "Job has been submitted with JobID" | sed 's/.* //g')
-
- wait_job_running ${JOB_ID}
-
- local c
- for (( c=0; c<${JM_KILLS}; c++ )); do
- # kill the JM and wait for watchdog to
- # create a new one which will take over
- kill_single 'StandaloneSessionClusterEntrypoint'
- wait_job_running ${JOB_ID}
- done
-
- for (( c=0; c<${TM_KILLS}; c++ )); do
- kill_and_replace_random_task_manager
- done
-
- touch ${LATCH_FILE_PATH}
-
- wait_job_terminal_state ${JOB_ID} "FINISHED"
- check_result_hash "DataSet-FineGrainedRecovery-Test" $TEST_DATA_DIR/out/dataset_allround "ac3d26e1afce19aa657527f000acb88b"
-}
-
-function setup_and_start_cluster() {
- local NUM_TASK_MANAGERS=$1
-
- create_ha_config
-
- set_config_key "jobmanager.execution.failover-strategy" "region"
- set_config_key "taskmanager.numberOfTaskSlots" "1"
-
- set_config_key "restart-strategy.type" "fixed-delay"
- set_config_key "restart-strategy.fixed-delay.attempts" "2147483647"
-
- set_config_key "heartbeat.interval" "2000"
- set_config_key "heartbeat.timeout" "10000"
-
- start_local_zk
- start_ha_jm_watchdog 1 "StandaloneSessionClusterEntrypoint" start_jm_cmd "8081"
- start_taskmanagers ${NUM_TASK_MANAGERS}
-}
-
-function kill_and_replace_random_task_manager() {
- local NUM_TASK_MANAGERS=$(query_number_of_running_tms)
-
- kill_random_taskmanager
- wait_for_number_of_running_tms $(( ${NUM_TASK_MANAGERS} - 1 ))
- start_taskmanagers 1
- wait_for_number_of_running_tms ${NUM_TASK_MANAGERS}
-}
-
-run_ha_test 4
diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_application.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_application.sh
index e0f001e11e54b..6ba5eca844ce1 100755
--- a/flink-end-to-end-tests/test-scripts/test_kubernetes_application.sh
+++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_application.sh
@@ -50,7 +50,7 @@ mkdir -p "$LOCAL_LOGS_PATH"
-Dkubernetes.jobmanager.cpu=0.5 \
-Dkubernetes.taskmanager.cpu=0.5 \
-Dkubernetes.rest-service.exposed.type=NodePort \
- local:///opt/flink/examples/batch/WordCount.jar
+ local:///opt/flink/examples/streaming/WordCount.jar
kubectl wait --for=condition=Available --timeout=30s deploy/${CLUSTER_ID} || exit 1
jm_pod_name=$(kubectl get pods --selector="app=${CLUSTER_ID},component=jobmanager" -o jsonpath='{..metadata.name}')
diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh
index 95c8dfe6da7ce..46d6e7032114d 100755
--- a/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh
+++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh
@@ -19,7 +19,7 @@
source "$(dirname "$0")"/common_kubernetes.sh
-export FLINK_JOB=org.apache.flink.examples.java.wordcount.WordCount
+export FLINK_JOB=org.apache.flink.streaming.examples.wordcount.WordCount
export FLINK_IMAGE_NAME=test_kubernetes_embedded_job
export OUTPUT_VOLUME=${TEST_DATA_DIR}/out
export OUTPUT_FILE=kubernetes_wc_out
@@ -44,7 +44,7 @@ if ! retry_times $IMAGE_BUILD_RETRIES $IMAGE_BUILD_BACKOFF "build_image ${FLINK_
exit 1
fi
-export USER_LIB=${FLINK_DIR}/examples/batch
+export USER_LIB=${FLINK_DIR}/examples/streaming
kubectl create -f ${CONTAINER_SCRIPTS}/job-cluster-service.yaml
envsubst '${FLINK_IMAGE_NAME} ${FLINK_JOB} ${FLINK_JOB_PARALLELISM} ${FLINK_JOB_ARGUMENTS} ${USER_LIB}' < ${CONTAINER_SCRIPTS}/job-cluster-job.yaml.template | kubectl create -f -
envsubst '${FLINK_IMAGE_NAME} ${FLINK_JOB_PARALLELISM} ${USER_LIB}' < ${CONTAINER_SCRIPTS}/task-manager-deployment.yaml.template | kubectl create -f -
diff --git a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh
index 0c30c0af63dd7..75fe46c12ee9c 100755
--- a/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh
+++ b/flink-end-to-end-tests/test-scripts/test_kubernetes_session.sh
@@ -25,6 +25,7 @@ FLINK_IMAGE_NAME="test_kubernetes_session"
LOCAL_OUTPUT_PATH="${TEST_DATA_DIR}/out/wc_out"
OUTPUT_PATH="/tmp/wc_out"
OUTPUT_ARGS="--output ${OUTPUT_PATH}"
+EXECUTION_MODE_ARGS="--execution-mode BATCH"
IMAGE_BUILD_RETRIES=3
IMAGE_BUILD_BACKOFF=2
@@ -32,12 +33,13 @@ INPUT_TYPE=${1:-embedded}
case $INPUT_TYPE in
(embedded)
INPUT_ARGS=""
+ RESULT_HASH="382f860d4fcf379b96b24f25857326eb"
;;
(dummy-fs)
source "$(dirname "$0")"/common_dummy_fs.sh
cp_dummy_fs_to_opt
INPUT_ARGS="--input dummy://localhost/words --input anotherDummy://localhost/words"
- RESULT_HASH="0e5bd0a3dd7d5a7110aa85ff70adb54b"
+ RESULT_HASH="41d097718a0b00f67fe13d21048d1757"
ENABLE_DUMMPY_FS_ARGS="-Dcontainerized.master.env.ENABLE_BUILT_IN_PLUGINS=flink-dummy-fs.jar;flink-another-dummy-fs.jar \
-Dcontainerized.taskmanager.env.ENABLE_BUILT_IN_PLUGINS=flink-dummy-fs.jar;flink-another-dummy-fs.jar"
;;
@@ -81,7 +83,7 @@ wait_rest_endpoint_up_k8s $jm_pod_name
"$FLINK_DIR"/bin/flink run -e kubernetes-session \
-Dkubernetes.cluster-id=${CLUSTER_ID} \
- ${FLINK_DIR}/examples/batch/WordCount.jar ${INPUT_ARGS} ${OUTPUT_ARGS}
+ ${FLINK_DIR}/examples/streaming/WordCount.jar ${INPUT_ARGS} ${OUTPUT_ARGS} ${EXECUTION_MODE_ARGS}
if ! check_logs_output $jm_pod_name 'Starting KubernetesSessionClusterEntrypoint'; then
echo "JobManager logs are not accessible via kubectl logs."
@@ -96,4 +98,5 @@ fi
kubectl cp `kubectl get pods | awk '/taskmanager/ {print $1}'`:${OUTPUT_PATH} ${LOCAL_OUTPUT_PATH}
-check_result_hash "WordCount" "${LOCAL_OUTPUT_PATH}" "${RESULT_HASH}"
+OUTPUT_FILES=$(find "${LOCAL_OUTPUT_PATH}" -type f)
+check_result_hash "WordCount" "${OUTPUT_FILES}" "${RESULT_HASH}"
diff --git a/flink-end-to-end-tests/test-scripts/test_nat.sh b/flink-end-to-end-tests/test-scripts/test_nat.sh
index f9129daa9f6a0..949894ebb27b4 100755
--- a/flink-end-to-end-tests/test-scripts/test_nat.sh
+++ b/flink-end-to-end-tests/test-scripts/test_nat.sh
@@ -24,7 +24,7 @@ DOCKER_SCRIPTS=${END_TO_END_DIR}/test-scripts/container-scripts
DOCKER_IMAGE_BUILD_RETRIES=3
BUILD_BACKOFF_TIME=5
-export FLINK_JOB=org.apache.flink.examples.java.wordcount.WordCount
+export FLINK_JOB=org.apache.flink.streaming.examples.wordcount.WordCount
export FLINK_DOCKER_IMAGE_NAME=test_nat
export INPUT_VOLUME=${END_TO_END_DIR}/test-scripts/test-data
export OUTPUT_VOLUME=${TEST_DATA_DIR}/out
@@ -47,11 +47,11 @@ export TM_1_DATA_EX_PORT=11001
export TM_2_DATA_EX_PORT=11002
export TM_DATA_IN_PORT=11000
-RESULT_HASH="72a690412be8928ba239c2da967328a5"
+RESULT_HASH="5a9945c9ab08890b2a0f6b31a4437d57"
INPUT_ARGS="--input ${INPUT_PATH}/words"
OUTPUT_PREFIX="docker_wc_out"
-export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/${OUTPUT_PREFIX}"
+export FLINK_JOB_ARGUMENTS="${INPUT_ARGS} --output ${OUTPUT_PATH}/${OUTPUT_PREFIX} --execution-mode BATCH"
# user inside the container must be able to create files, this is a workaround in-container permissions
mkdir -p $OUTPUT_VOLUME
@@ -63,11 +63,12 @@ if ! retry_times $DOCKER_IMAGE_BUILD_RETRIES ${BUILD_BACKOFF_TIME} "build_image
fi
popd
-export USER_LIB=${FLINK_DIR}/examples/batch
+export USER_LIB=${FLINK_DIR}/examples/streaming
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml up --force-recreate --abort-on-container-exit --exit-code-from job-cluster &> /dev/null
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml logs job-cluster > $FLINK_LOG_DIR/jobmanager.log
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml logs taskmanager1 > $FLINK_LOG_DIR/taskmanager1.log
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml logs taskmanager2 > $FLINK_LOG_DIR/taskmanager2.log
docker compose -f ${DOCKER_SCRIPTS}/docker-compose.nat.yml rm -f
-check_result_hash "WordCount" ${OUTPUT_VOLUME}/${OUTPUT_PREFIX}/ "${RESULT_HASH}"
+OUTPUT_FILES=$(find "${OUTPUT_VOLUME}/${OUTPUT_PREFIX}/" -type f)
+check_result_hash "WordCount" "${OUTPUT_FILES}" "${RESULT_HASH}"
diff --git a/flink-examples/flink-examples-batch/pom.xml b/flink-examples/flink-examples-batch/pom.xml
deleted file mode 100644
index ab0ff4b86924a..0000000000000
--- a/flink-examples/flink-examples-batch/pom.xml
+++ /dev/null
@@ -1,304 +0,0 @@
-
-
-
-
- 4.0.0
-
-
- org.apache.flink
- flink-examples
- 2.0-SNAPSHOT
-
-
- flink-examples-batch
- Flink : Examples : Batch
- jar
-
-
-
- org.apache.flink
- flink-java
- ${project.version}
-
-
-
- org.apache.flink
- flink-clients
- ${project.version}
-
-
-
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-jar-plugin
-
-
-
-
- KMeans
- package
-
- jar
-
-
-
- KMeans
-
-
-
- org.apache.flink.examples.java.clustering.KMeans
-
-
-
-
- **/java/clustering/KMeans.class
- **/java/clustering/KMeans$*.class
- **/java/clustering/util/KMeansDataGenerator.class
- **/java/clustering/util/KMeansData.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- TransitiveClosure
- package
-
- jar
-
-
- TransitiveClosure
-
-
-
- org.apache.flink.examples.java.graph.TransitiveClosureNaive
-
-
-
-
- **/java/graph/TransitiveClosureNaive.class
- **/java/graph/TransitiveClosureNaive$*.class
- **/java/graph/util/ConnectedComponentsData.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- ConnectedComponents
- package
-
- jar
-
-
- ConnectedComponents
-
-
-
- org.apache.flink.examples.java.graph.ConnectedComponents
-
-
-
-
- **/java/graph/ConnectedComponents.class
- **/java/graph/ConnectedComponents$*.class
- **/java/graph/util/ConnectedComponentsData.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- EnumTriangles
- package
-
- jar
-
-
- EnumTriangles
-
-
-
- org.apache.flink.examples.java.graph.EnumTriangles
-
-
-
-
- **/java/graph/EnumTriangles.class
- **/java/graph/EnumTriangles$*.class
- **/java/graph/util/EnumTrianglesDataTypes.class
- **/java/graph/util/EnumTrianglesDataTypes$*.class
- **/java/graph/util/EnumTrianglesData.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- PageRank
- package
-
- jar
-
-
- PageRank
-
-
-
- org.apache.flink.examples.java.graph.PageRank
-
-
-
-
- **/java/graph/PageRank.class
- **/java/graph/PageRank$*.class
- **/java/graph/util/PageRankData.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- WebLogAnalysis
- package
-
- jar
-
-
- WebLogAnalysis
-
-
-
- org.apache.flink.examples.java.relational.WebLogAnalysis
-
-
-
-
- **/java/relational/WebLogAnalysis.class
- **/java/relational/WebLogAnalysis$*.class
- **/java/relational/util/WebLogData.class
- **/java/relational/util/WebLogDataGenerator.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- WordCount
- package
-
- jar
-
-
- WordCount
-
-
-
- org.apache.flink.examples.java.wordcount.WordCount
-
-
-
-
- **/java/wordcount/WordCount.class
- **/java/wordcount/WordCount$*.class
- **/java/wordcount/util/WordCountData.class
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
- DistCp
- package
-
- jar
-
-
- DistCp
-
-
-
- org.apache.flink.examples.java.distcp.DistCp
-
-
-
-
- **/java/distcp/*
- **/java/util/DataSetDeprecationInfo.class
- META-INF/LICENSE
- META-INF/NOTICE
-
-
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-antrun-plugin
-
-
- rename
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
-
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
deleted file mode 100644
index a5aae3d90772a..0000000000000
--- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/KMeans.java
+++ /dev/null
@@ -1,321 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.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;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.api.java.functions.FunctionAnnotation.ForwardedFields;
-import org.apache.flink.api.java.operators.IterativeDataSet;
-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.examples.java.clustering.util.KMeansData;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.Serializable;
-import java.util.Collection;
-
-import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO;
-
-/**
- * This example implements a basic K-Means clustering algorithm.
- *
- *
K-Means is an iterative clustering algorithm and works as follows:
- * K-Means is given a set of data points to be clustered and an initial set of K cluster
- * centers. In each iteration, the algorithm computes the distance of each data point to each
- * cluster center. Each point is assigned to the cluster center which is closest to it.
- * Subsequently, each cluster center is moved to the center (mean) of all points that have
- * been assigned to it. The moved cluster centers are fed into the next iteration. The algorithm
- * terminates after a fixed number of iterations (as in this implementation) or if cluster centers
- * do not (significantly) move in an iteration.
- * This is the Wikipedia entry for the K-Means Clustering algorithm.
- *
- *
This implementation works on two-dimensional data points.
- * It computes an assignment of data points to cluster centers, i.e., each data point is annotated
- * with the id of the final cluster (center) it belongs to.
- *
- *
Input files are plain text files and must be formatted as follows:
- *
- *
- *
Data points are represented as two double values separated by a blank character. Data
- * points are separated by newline characters.
- * For example "1.2 2.3\n5.3 7.2\n" gives two data points (x=1.2, y=2.3) and
- * (x=5.3, y=7.2).
- *
Cluster centers are represented by an integer id and a point value.
- * For example "1 6.2 3.2\n2 2.9 5.7\n" gives two centers (id=1, x=6.2, y=3.2)
- * and (id=2, x=2.9, y=5.7).
- *
- *
- *
Usage:
- * KMeans --points <path> --centroids <path> --output <path> --iterations <n>
- *
- * If no parameters are provided, the program is run with default data from {@link
- * org.apache.flink.examples.java.clustering.util.KMeansData} and 10 iterations.
- *
- *
This example shows how to use:
- *
- *
- *
Bulk iterations
- *
Broadcast variables in bulk iterations
- *
Custom Java objects (POJOs)
- *
- *
- *
Note: All Flink DataSet APIs are deprecated since Flink 1.18 and will be removed in a future
- * Flink major version. You can still build your application in DataSet, but you should move to
- * either the DataStream and/or Table API. This class is retained for testing purposes.
- */
-@SuppressWarnings("serial")
-public class KMeans {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(KMeans.class);
-
- public static void main(String[] args) throws Exception {
-
- LOGGER.warn(DATASET_DEPRECATION_INFO);
-
- // Checking input parameters
- final ParameterTool params = ParameterTool.fromArgs(args);
-
- // set up execution environment
- ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
- env.getConfig()
- .setGlobalJobParameters(params); // make parameters available in the web interface
-
- // get input data:
- // read the points and centroids from the provided paths or fall back to default data
- DataSet points = getPointDataSet(params, env);
- DataSet centroids = getCentroidDataSet(params, env);
-
- // set number of bulk iterations for KMeans algorithm
- IterativeDataSet loop = centroids.iterate(params.getInt("iterations", 10));
-
- DataSet newCentroids =
- points
- // compute closest centroid for each point
- .map(new SelectNearestCenter())
- .withBroadcastSet(loop, "centroids")
- // count and sum point coordinates for each centroid
- .map(new CountAppender())
- .groupBy(0)
- .reduce(new CentroidAccumulator())
- // compute new centroids from point counts and coordinate sums
- .map(new CentroidAverager());
-
- // feed new centroids back into next iteration
- DataSet finalCentroids = loop.closeWith(newCentroids);
-
- DataSet> clusteredPoints =
- points
- // assign points to final clusters
- .map(new SelectNearestCenter())
- .withBroadcastSet(finalCentroids, "centroids");
-
- // emit result
- if (params.has("output")) {
- clusteredPoints.writeAsCsv(params.get("output"), "\n", " ");
-
- // since file sinks are lazy, we trigger the execution explicitly
- env.execute("KMeans Example");
- } else {
- System.out.println("Printing result to stdout. Use --output to specify output path.");
- clusteredPoints.print();
- }
- }
-
- // *************************************************************************
- // DATA SOURCE READING (POINTS AND CENTROIDS)
- // *************************************************************************
-
- private static DataSet getCentroidDataSet(
- ParameterTool params, ExecutionEnvironment env) {
- DataSet centroids;
- if (params.has("centroids")) {
- centroids =
- env.readCsvFile(params.get("centroids"))
- .fieldDelimiter(" ")
- .pojoType(Centroid.class, "id", "x", "y");
- } else {
- System.out.println("Executing K-Means example with default centroid data set.");
- System.out.println("Use --centroids to specify file input.");
- centroids = KMeansData.getDefaultCentroidDataSet(env);
- }
- return centroids;
- }
-
- private static DataSet getPointDataSet(ParameterTool params, ExecutionEnvironment env) {
- DataSet points;
- if (params.has("points")) {
- // read points from CSV file
- points =
- env.readCsvFile(params.get("points"))
- .fieldDelimiter(" ")
- .pojoType(Point.class, "x", "y");
- } else {
- System.out.println("Executing K-Means example with default point data set.");
- System.out.println("Use --points to specify file input.");
- points = KMeansData.getDefaultPointDataSet(env);
- }
- return points;
- }
-
- // *************************************************************************
- // DATA TYPES
- // *************************************************************************
-
- /** A simple two-dimensional point. */
- public static class Point implements Serializable {
-
- public double x, y;
-
- public Point() {}
-
- public Point(double x, double y) {
- this.x = x;
- this.y = y;
- }
-
- public Point add(Point other) {
- x += other.x;
- y += other.y;
- return this;
- }
-
- public Point div(long val) {
- x /= val;
- y /= val;
- return this;
- }
-
- public double euclideanDistance(Point other) {
- return Math.sqrt((x - other.x) * (x - other.x) + (y - other.y) * (y - other.y));
- }
-
- public void clear() {
- x = y = 0.0;
- }
-
- @Override
- public String toString() {
- return x + " " + y;
- }
- }
-
- /** A simple two-dimensional centroid, basically a point with an ID. */
- public static class Centroid extends Point {
-
- public int id;
-
- public Centroid() {}
-
- public Centroid(int id, double x, double y) {
- super(x, y);
- this.id = id;
- }
-
- public Centroid(int id, Point p) {
- super(p.x, p.y);
- this.id = id;
- }
-
- @Override
- public String toString() {
- return id + " " + super.toString();
- }
- }
-
- // *************************************************************************
- // USER FUNCTIONS
- // *************************************************************************
-
- /** Determines the closest cluster center for a data point. */
- @ForwardedFields("*->1")
- public static final class SelectNearestCenter
- extends RichMapFunction> {
- private Collection centroids;
-
- /** Reads the centroid values from a broadcast variable into a collection. */
- @Override
- public void open(OpenContext openContext) throws Exception {
- this.centroids = getRuntimeContext().getBroadcastVariable("centroids");
- }
-
- @Override
- public Tuple2 map(Point p) throws Exception {
-
- double minDistance = Double.MAX_VALUE;
- int closestCentroidId = -1;
-
- // check all cluster centers
- for (Centroid centroid : centroids) {
- // compute distance
- double distance = p.euclideanDistance(centroid);
-
- // update nearest cluster if necessary
- if (distance < minDistance) {
- minDistance = distance;
- closestCentroidId = centroid.id;
- }
- }
-
- // emit a new record with the center id and the data point.
- return new Tuple2<>(closestCentroidId, p);
- }
- }
-
- /** Appends a count variable to the tuple. */
- @ForwardedFields("f0;f1")
- public static final class CountAppender
- implements MapFunction, Tuple3> {
-
- @Override
- public Tuple3 map(Tuple2 t) {
- return new Tuple3<>(t.f0, t.f1, 1L);
- }
- }
-
- /** Sums and counts point coordinates. */
- @ForwardedFields("0")
- public static final class CentroidAccumulator
- implements ReduceFunction> {
-
- @Override
- public Tuple3 reduce(
- Tuple3 val1, Tuple3 val2) {
- return new Tuple3<>(val1.f0, val1.f1.add(val2.f1), val1.f2 + val2.f2);
- }
- }
-
- /** Computes new centroid from coordinate sum and count of points. */
- @ForwardedFields("0->id")
- public static final class CentroidAverager
- implements MapFunction, Centroid> {
-
- @Override
- public Centroid map(Tuple3 value) {
- return new Centroid(value.f0, value.f1.div(value.f2));
- }
- }
-}
diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java
deleted file mode 100644
index 9bd1e182e7595..0000000000000
--- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansData.java
+++ /dev/null
@@ -1,105 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.examples.java.clustering.util;
-
-import org.apache.flink.api.java.DataSet;
-import org.apache.flink.api.java.ExecutionEnvironment;
-import org.apache.flink.examples.java.clustering.KMeans.Centroid;
-import org.apache.flink.examples.java.clustering.KMeans.Point;
-
-import java.util.LinkedList;
-import java.util.List;
-
-/**
- * Provides the default data sets used for the K-Means example program. The default data sets are
- * used, if no parameters are given to the program.
- */
-public class KMeansData {
-
- // We have the data as object arrays so that we can also generate Scala Data Sources from it.
- public static final Object[][] CENTROIDS =
- new Object[][] {
- new Object[] {1, -31.85, -44.77},
- new Object[] {2, 35.16, 17.46},
- new Object[] {3, -5.16, 21.93},
- new Object[] {4, -24.06, 6.81}
- };
-
- public static final Object[][] POINTS =
- new Object[][] {
- new Object[] {-14.22, -48.01},
- new Object[] {-22.78, 37.10},
- new Object[] {56.18, -42.99},
- new Object[] {35.04, 50.29},
- new Object[] {-9.53, -46.26},
- new Object[] {-34.35, 48.25},
- new Object[] {55.82, -57.49},
- new Object[] {21.03, 54.64},
- new Object[] {-13.63, -42.26},
- new Object[] {-36.57, 32.63},
- new Object[] {50.65, -52.40},
- new Object[] {24.48, 34.04},
- new Object[] {-2.69, -36.02},
- new Object[] {-38.80, 36.58},
- new Object[] {24.00, -53.74},
- new Object[] {32.41, 24.96},
- new Object[] {-4.32, -56.92},
- new Object[] {-22.68, 29.42},
- new Object[] {59.02, -39.56},
- new Object[] {24.47, 45.07},
- new Object[] {5.23, -41.20},
- new Object[] {-23.00, 38.15},
- new Object[] {44.55, -51.50},
- new Object[] {14.62, 59.06},
- new Object[] {7.41, -56.05},
- new Object[] {-26.63, 28.97},
- new Object[] {47.37, -44.72},
- new Object[] {29.07, 51.06},
- new Object[] {0.59, -31.89},
- new Object[] {-39.09, 20.78},
- new Object[] {42.97, -48.98},
- new Object[] {34.36, 49.08},
- new Object[] {-21.91, -49.01},
- new Object[] {-46.68, 46.04},
- new Object[] {48.52, -43.67},
- new Object[] {30.05, 49.25},
- new Object[] {4.03, -43.56},
- new Object[] {-37.85, 41.72},
- new Object[] {38.24, -48.32},
- new Object[] {20.83, 57.85}
- };
-
- public static DataSet getDefaultCentroidDataSet(ExecutionEnvironment env) {
- List centroidList = new LinkedList();
- for (Object[] centroid : CENTROIDS) {
- centroidList.add(
- new Centroid(
- (Integer) centroid[0], (Double) centroid[1], (Double) centroid[2]));
- }
- return env.fromCollection(centroidList);
- }
-
- public static DataSet getDefaultPointDataSet(ExecutionEnvironment env) {
- List pointList = new LinkedList();
- for (Object[] point : POINTS) {
- pointList.add(new Point((Double) point[0], (Double) point[1]));
- }
- return env.fromCollection(pointList);
- }
-}
diff --git a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java b/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
deleted file mode 100644
index 348192af18793..0000000000000
--- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/clustering/util/KMeansDataGenerator.java
+++ /dev/null
@@ -1,190 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.examples.java.clustering.util;
-
-import org.apache.flink.api.java.utils.ParameterTool;
-import org.apache.flink.examples.java.clustering.KMeans;
-
-import java.io.BufferedWriter;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.text.DecimalFormat;
-import java.util.Locale;
-import java.util.Random;
-
-/** Generates data for the {@link KMeans} example program. */
-public class KMeansDataGenerator {
-
- static {
- Locale.setDefault(Locale.US);
- }
-
- private static final String CENTERS_FILE = "centers";
- private static final String POINTS_FILE = "points";
- private static final long DEFAULT_SEED = 4650285087650871364L;
- private static final double DEFAULT_VALUE_RANGE = 100.0;
- private static final double RELATIVE_STDDEV = 0.08;
- private static final int DIMENSIONALITY = 2;
- private static final DecimalFormat FORMAT = new DecimalFormat("#0.00");
- private static final char DELIMITER = ' ';
-
- /**
- * Main method to generate data for the {@link KMeans} example program.
- *
- *
The generator creates to files:
- *
- *
- *
< output-path >/points for the data points
- *
< output-path >/centers for the cluster centers
- *
- *
- * @param args
- *
- *
Int: Number of data points
- *
Int: Number of cluster centers
- *
Optional String: Output path, default value is {tmp.dir}
- *
Optional Double: Standard deviation of data points
- *
Optional Double: Value range of cluster centers
- *
Optional Long: Random seed
- *
- *
- * @throws IOException
- */
- public static void main(String[] args) throws IOException {
-
- // check parameter count
- if (args.length < 2) {
- System.out.println(
- "KMeansDataGenerator -points -k [-output ] [-stddev ] [-range ] [-seed ]");
- System.exit(1);
- }
-
- // parse parameters
-
- final ParameterTool params = ParameterTool.fromArgs(args);
- final int numDataPoints = params.getInt("points");
- final int k = params.getInt("k");
- final String outDir = params.get("output", System.getProperty("java.io.tmpdir"));
- final double stddev = params.getDouble("stddev", RELATIVE_STDDEV);
- final double range = params.getDouble("range", DEFAULT_VALUE_RANGE);
- final long firstSeed = params.getLong("seed", DEFAULT_SEED);
-
- final double absoluteStdDev = stddev * range;
- final Random random = new Random(firstSeed);
-
- // the means around which data points are distributed
- final double[][] means = uniformRandomCenters(random, k, DIMENSIONALITY, range);
-
- // write the points out
- BufferedWriter pointsOut = null;
- try {
- pointsOut = new BufferedWriter(new FileWriter(new File(outDir + "/" + POINTS_FILE)));
- StringBuilder buffer = new StringBuilder();
-
- double[] point = new double[DIMENSIONALITY];
- int nextCentroid = 0;
-
- for (int i = 1; i <= numDataPoints; i++) {
- // generate a point for the current centroid
- double[] centroid = means[nextCentroid];
- for (int d = 0; d < DIMENSIONALITY; d++) {
- point[d] = (random.nextGaussian() * absoluteStdDev) + centroid[d];
- }
- writePoint(point, buffer, pointsOut);
- nextCentroid = (nextCentroid + 1) % k;
- }
- } finally {
- if (pointsOut != null) {
- pointsOut.close();
- }
- }
-
- // write the uniformly distributed centers to a file
- BufferedWriter centersOut = null;
- try {
- centersOut = new BufferedWriter(new FileWriter(new File(outDir + "/" + CENTERS_FILE)));
- StringBuilder buffer = new StringBuilder();
-
- double[][] centers = uniformRandomCenters(random, k, DIMENSIONALITY, range);
-
- for (int i = 0; i < k; i++) {
- writeCenter(i + 1, centers[i], buffer, centersOut);
- }
- } finally {
- if (centersOut != null) {
- centersOut.close();
- }
- }
-
- System.out.println(
- "Wrote " + numDataPoints + " data points to " + outDir + "/" + POINTS_FILE);
- System.out.println("Wrote " + k + " cluster centers to " + outDir + "/" + CENTERS_FILE);
- }
-
- private static double[][] uniformRandomCenters(
- Random rnd, int num, int dimensionality, double range) {
- final double halfRange = range / 2;
- final double[][] points = new double[num][dimensionality];
-
- for (int i = 0; i < num; i++) {
- for (int dim = 0; dim < dimensionality; dim++) {
- points[i][dim] = (rnd.nextDouble() * range) - halfRange;
- }
- }
- return points;
- }
-
- private static void writePoint(double[] coordinates, StringBuilder buffer, BufferedWriter out)
- throws IOException {
- buffer.setLength(0);
-
- // write coordinates
- for (int j = 0; j < coordinates.length; j++) {
- buffer.append(FORMAT.format(coordinates[j]));
- if (j < coordinates.length - 1) {
- buffer.append(DELIMITER);
- }
- }
-
- out.write(buffer.toString());
- out.newLine();
- }
-
- private static void writeCenter(
- long id, double[] coordinates, StringBuilder buffer, BufferedWriter out)
- throws IOException {
- buffer.setLength(0);
-
- // write id
- buffer.append(id);
- buffer.append(DELIMITER);
-
- // write coordinates
- for (int j = 0; j < coordinates.length; j++) {
- buffer.append(FORMAT.format(coordinates[j]));
- if (j < coordinates.length - 1) {
- buffer.append(DELIMITER);
- }
- }
-
- out.write(buffer.toString());
- out.newLine();
- }
-}
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
deleted file mode 100644
index d370148c27149..0000000000000
--- a/flink-examples/flink-examples-batch/src/main/java/org/apache/flink/examples/java/distcp/DistCp.java
+++ /dev/null
@@ -1,212 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.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;
-import org.apache.flink.api.java.LocalEnvironment;
-import org.apache.flink.api.java.operators.DataSource;
-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.core.fs.FSDataInputStream;
-import org.apache.flink.core.fs.FSDataOutputStream;
-import org.apache.flink.core.fs.FileStatus;
-import org.apache.flink.core.fs.FileSystem;
-import org.apache.flink.core.fs.Path;
-import org.apache.flink.util.Collector;
-
-import org.apache.commons.io.IOUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.flink.examples.java.util.DataSetDeprecationInfo.DATASET_DEPRECATION_INFO;
-
-/**
- * A main class of the Flink distcp utility. It's a simple reimplementation of Hadoop distcp (see http://hadoop.apache.org/docs/r1.2.1/distcp.html)
- * with a dynamic input format Note that this tool does not deal with retries. Additionally, empty
- * directories are not copied over.
- *
- *
When running locally, local file systems paths can be used. However, in a distributed
- * environment HDFS paths must be provided both as input and output.
- *
- *
Note: All Flink DataSet APIs are deprecated since Flink 1.18 and will be removed in a future
- * Flink major version. You can still build your application in DataSet, but you should move to
- * either the DataStream and/or Table API. This class is retained for testing purposes.
- */
-public class DistCp {
-
- private static final Logger LOGGER = LoggerFactory.getLogger(DistCp.class);
- public static final String BYTES_COPIED_CNT_NAME = "BYTES_COPIED";
- public static final String FILES_COPIED_CNT_NAME = "FILES_COPIED";
-
- public static void main(String[] args) throws Exception {
-
- LOGGER.warn(DATASET_DEPRECATION_INFO);
-
- // set up the execution environment
- final ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment();
-
- ParameterTool params = ParameterTool.fromArgs(args);
- if (!params.has("input") || !params.has("output")) {
- System.err.println("Usage: --input --output [--parallelism ]");
- return;
- }
-
- final Path sourcePath = new Path(params.get("input"));
- final Path targetPath = new Path(params.get("output"));
- if (!isLocal(env) && !(isOnDistributedFS(sourcePath) && isOnDistributedFS(targetPath))) {
- System.out.println("In a distributed mode only HDFS input/output paths are supported");
- return;
- }
-
- final int parallelism = params.getInt("parallelism", 10);
- if (parallelism <= 0) {
- System.err.println("Parallelism should be greater than 0");
- return;
- }
-
- // make parameters available in the web interface
- env.getConfig().setGlobalJobParameters(params);
-
- env.setParallelism(parallelism);
-
- long startTime = System.currentTimeMillis();
- LOGGER.info("Initializing copy tasks");
- List tasks = getCopyTasks(sourcePath);
- LOGGER.info(
- "Copy task initialization took " + (System.currentTimeMillis() - startTime) + "ms");
-
- DataSet inputTasks =
- new DataSource<>(
- env,
- new FileCopyTaskInputFormat(tasks),
- new GenericTypeInfo<>(FileCopyTask.class),
- "fileCopyTasks");
-
- FlatMapOperator res =
- inputTasks.flatMap(
- new RichFlatMapFunction() {
-
- private static final long serialVersionUID = 1109254230243989929L;
- private LongCounter fileCounter;
- private LongCounter bytesCounter;
-
- @Override
- public void open(OpenContext openContext) throws Exception {
- bytesCounter =
- getRuntimeContext().getLongCounter(BYTES_COPIED_CNT_NAME);
- fileCounter =
- getRuntimeContext().getLongCounter(FILES_COPIED_CNT_NAME);
- }
-
- @Override
- public void flatMap(FileCopyTask task, Collector